2013-11-19 15:53:40 +01:00
|
|
|
/**
|
2015-03-07 22:58:48 -08:00
|
|
|
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
2013-11-19 15:53:40 +01:00
|
|
|
*/
|
2015-04-27 14:48:28 +02:00
|
|
|
package akka.cluster.sharding
|
2013-11-19 15:53:40 +01:00
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
import akka.cluster.ddata.{ ReplicatorSettings, Replicator }
|
2015-04-27 14:48:28 +02:00
|
|
|
import akka.cluster.sharding.ShardCoordinator.Internal.{ ShardStopped, HandOff }
|
|
|
|
|
import akka.cluster.sharding.ShardRegion.Passivate
|
2015-06-08 11:06:16 +02:00
|
|
|
import akka.cluster.sharding.ShardRegion.GetCurrentRegions
|
|
|
|
|
import akka.cluster.sharding.ShardRegion.CurrentRegions
|
2013-11-19 15:53:40 +01:00
|
|
|
import language.postfixOps
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
2014-08-21 16:39:24 +01:00
|
|
|
import akka.actor._
|
2013-11-19 15:53:40 +01:00
|
|
|
import akka.cluster.Cluster
|
2014-05-21 01:35:21 +02:00
|
|
|
import akka.persistence.PersistentActor
|
2013-11-19 15:53:40 +01: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 akka.testkit.TestEvent.Mute
|
|
|
|
|
import java.io.File
|
|
|
|
|
import org.apache.commons.io.FileUtils
|
2015-04-27 14:48:28 +02:00
|
|
|
import akka.cluster.singleton.ClusterSingletonManager
|
2015-04-29 18:23:45 +02:00
|
|
|
import akka.cluster.singleton.ClusterSingletonManagerSettings
|
2015-07-07 16:28:17 +02:00
|
|
|
import akka.pattern.BackoffSupervisor
|
2013-11-19 15:53:40 +01:00
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
object ClusterShardingSpec {
|
2013-11-19 15:53:40 +01:00
|
|
|
//#counter-actor
|
|
|
|
|
case object Increment
|
|
|
|
|
case object Decrement
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Get(counterId: Long)
|
2015-06-09 12:25:58 +02:00
|
|
|
final case class EntityEnvelope(id: Long, payload: Any)
|
2013-11-19 15:53:40 +01:00
|
|
|
|
|
|
|
|
case object Stop
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class CounterChanged(delta: Int)
|
2013-11-19 15:53:40 +01:00
|
|
|
|
2014-05-21 01:35:21 +02:00
|
|
|
class Counter extends PersistentActor {
|
2013-11-19 15:53:40 +01:00
|
|
|
import ShardRegion.Passivate
|
|
|
|
|
|
|
|
|
|
context.setReceiveTimeout(120.seconds)
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
// self.path.name is the entity identifier (utf-8 URL-encoded)
|
2015-06-30 11:43:37 +02:00
|
|
|
override def persistenceId: String = "Counter-" + self.path.name
|
2014-06-26 13:56:01 +02:00
|
|
|
|
2013-11-19 15:53:40 +01:00
|
|
|
var count = 0
|
|
|
|
|
//#counter-actor
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
super.postStop()
|
2015-06-02 21:01:00 -07:00
|
|
|
// Simulate that the passivation takes some time, to verify passivation buffering
|
2013-11-19 15:53:40 +01:00
|
|
|
Thread.sleep(500)
|
|
|
|
|
}
|
|
|
|
|
//#counter-actor
|
|
|
|
|
|
|
|
|
|
def updateState(event: CounterChanged): Unit =
|
|
|
|
|
count += event.delta
|
|
|
|
|
|
2014-01-19 17:46:32 +01:00
|
|
|
override def receiveRecover: Receive = {
|
2013-11-19 15:53:40 +01:00
|
|
|
case evt: CounterChanged ⇒ updateState(evt)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def receiveCommand: Receive = {
|
|
|
|
|
case Increment ⇒ persist(CounterChanged(+1))(updateState)
|
|
|
|
|
case Decrement ⇒ persist(CounterChanged(-1))(updateState)
|
2014-01-16 15:16:35 +01:00
|
|
|
case Get(_) ⇒ sender() ! count
|
2013-11-19 15:53:40 +01:00
|
|
|
case ReceiveTimeout ⇒ context.parent ! Passivate(stopMessage = Stop)
|
|
|
|
|
case Stop ⇒ context.stop(self)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
//#counter-actor
|
|
|
|
|
|
2015-06-09 16:02:19 +02:00
|
|
|
val extractEntityId: ShardRegion.ExtractEntityId = {
|
2015-06-09 12:25:58 +02:00
|
|
|
case EntityEnvelope(id, payload) ⇒ (id.toString, payload)
|
|
|
|
|
case msg @ Get(id) ⇒ (id.toString, msg)
|
2014-10-27 10:52:34 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val numberOfShards = 12
|
|
|
|
|
|
2015-06-09 16:02:19 +02:00
|
|
|
val extractShardId: ShardRegion.ExtractShardId = {
|
2015-06-09 12:25:58 +02:00
|
|
|
case EntityEnvelope(id, _) ⇒ (id % numberOfShards).toString
|
|
|
|
|
case Get(id) ⇒ (id % numberOfShards).toString
|
2014-10-27 10:52:34 +01:00
|
|
|
}
|
|
|
|
|
|
2015-06-30 11:43:37 +02:00
|
|
|
def qualifiedCounterProps(typeName: String): Props =
|
|
|
|
|
Props(new QualifiedCounter(typeName))
|
|
|
|
|
|
|
|
|
|
class QualifiedCounter(typeName: String) extends Counter {
|
|
|
|
|
override def persistenceId: String = typeName + "-" + self.path.name
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class AnotherCounter extends QualifiedCounter("AnotherCounter")
|
|
|
|
|
|
2014-10-27 10:52:34 +01:00
|
|
|
}
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
abstract class ClusterShardingSpecConfig(val mode: String) extends MultiNodeConfig {
|
|
|
|
|
val controller = role("controller")
|
|
|
|
|
val first = role("first")
|
|
|
|
|
val second = role("second")
|
|
|
|
|
val third = role("third")
|
|
|
|
|
val fourth = role("fourth")
|
|
|
|
|
val fifth = role("fifth")
|
|
|
|
|
val sixth = role("sixth")
|
|
|
|
|
|
|
|
|
|
commonConfig(ConfigFactory.parseString(s"""
|
|
|
|
|
akka.loglevel = INFO
|
|
|
|
|
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
|
|
|
|
akka.remote.log-remote-lifecycle-events = off
|
|
|
|
|
akka.cluster.auto-down-unreachable-after = 0s
|
|
|
|
|
akka.cluster.roles = ["backend"]
|
|
|
|
|
akka.cluster.distributed-data.gossip-interval = 1s
|
|
|
|
|
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"
|
|
|
|
|
akka.persistence.journal.leveldb-shared.store {
|
|
|
|
|
native = off
|
|
|
|
|
dir = "target/journal-ClusterShardingSpec"
|
|
|
|
|
}
|
|
|
|
|
akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
|
|
|
|
|
akka.persistence.snapshot-store.local.dir = "target/snapshots-ClusterShardingSpec"
|
|
|
|
|
akka.cluster.sharding {
|
|
|
|
|
retry-interval = 1 s
|
|
|
|
|
handoff-timeout = 10 s
|
|
|
|
|
shard-start-timeout = 5s
|
|
|
|
|
entity-restart-backoff = 1s
|
|
|
|
|
rebalance-interval = 2 s
|
|
|
|
|
state-store-mode = "$mode"
|
|
|
|
|
least-shard-allocation-strategy {
|
|
|
|
|
rebalance-threshold = 2
|
|
|
|
|
max-simultaneous-rebalance = 1
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
"""))
|
|
|
|
|
nodeConfig(sixth) {
|
|
|
|
|
ConfigFactory.parseString("""akka.cluster.roles = ["frontend"]""")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-27 10:52:34 +01:00
|
|
|
// only used in documentation
|
|
|
|
|
object ClusterShardingDocCode {
|
|
|
|
|
import ClusterShardingSpec._
|
|
|
|
|
|
2013-11-19 15:53:40 +01:00
|
|
|
//#counter-extractor
|
2015-06-09 16:02:19 +02:00
|
|
|
val extractEntityId: ShardRegion.ExtractEntityId = {
|
2015-06-09 12:25:58 +02:00
|
|
|
case EntityEnvelope(id, payload) ⇒ (id.toString, payload)
|
|
|
|
|
case msg @ Get(id) ⇒ (id.toString, msg)
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
2014-10-27 10:52:34 +01:00
|
|
|
val numberOfShards = 100
|
|
|
|
|
|
2015-06-09 16:02:19 +02:00
|
|
|
val extractShardId: ShardRegion.ExtractShardId = {
|
2015-06-09 12:25:58 +02:00
|
|
|
case EntityEnvelope(id, _) ⇒ (id % numberOfShards).toString
|
|
|
|
|
case Get(id) ⇒ (id % numberOfShards).toString
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
//#counter-extractor
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
object PersistentClusterShardingSpecConfig extends ClusterShardingSpecConfig("persistence")
|
|
|
|
|
object DDataClusterShardingSpecConfig extends ClusterShardingSpecConfig("ddata")
|
|
|
|
|
|
|
|
|
|
class PersistentClusterShardingSpec extends ClusterShardingSpec(PersistentClusterShardingSpecConfig)
|
|
|
|
|
class DDataClusterShardingSpec extends ClusterShardingSpec(DDataClusterShardingSpecConfig)
|
|
|
|
|
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode1 extends PersistentClusterShardingSpec
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode2 extends PersistentClusterShardingSpec
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode3 extends PersistentClusterShardingSpec
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode4 extends PersistentClusterShardingSpec
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode5 extends PersistentClusterShardingSpec
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode6 extends PersistentClusterShardingSpec
|
|
|
|
|
class PersistentClusterShardingMultiJvmNode7 extends PersistentClusterShardingSpec
|
|
|
|
|
|
|
|
|
|
class DDataClusterShardingMultiJvmNode1 extends DDataClusterShardingSpec
|
|
|
|
|
class DDataClusterShardingMultiJvmNode2 extends DDataClusterShardingSpec
|
|
|
|
|
class DDataClusterShardingMultiJvmNode3 extends DDataClusterShardingSpec
|
|
|
|
|
class DDataClusterShardingMultiJvmNode4 extends DDataClusterShardingSpec
|
|
|
|
|
class DDataClusterShardingMultiJvmNode5 extends DDataClusterShardingSpec
|
|
|
|
|
class DDataClusterShardingMultiJvmNode6 extends DDataClusterShardingSpec
|
|
|
|
|
class DDataClusterShardingMultiJvmNode7 extends DDataClusterShardingSpec
|
|
|
|
|
|
|
|
|
|
abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends MultiNodeSpec(config) with STMultiNodeSpec with ImplicitSender {
|
2013-11-19 15:53:40 +01:00
|
|
|
import ClusterShardingSpec._
|
2015-08-20 13:24:39 +03:00
|
|
|
import config._
|
2013-11-19 15:53:40 +01:00
|
|
|
|
|
|
|
|
override def initialParticipants = roles.size
|
|
|
|
|
|
|
|
|
|
val storageLocations = List(
|
|
|
|
|
"akka.persistence.journal.leveldb.dir",
|
|
|
|
|
"akka.persistence.journal.leveldb-shared.store.dir",
|
|
|
|
|
"akka.persistence.snapshot-store.local.dir").map(s ⇒ new File(system.settings.config.getString(s)))
|
|
|
|
|
|
|
|
|
|
override protected def atStartup() {
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
storageLocations.foreach(dir ⇒ if (dir.exists) FileUtils.deleteDirectory(dir))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override protected def afterTermination() {
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
storageLocations.foreach(dir ⇒ if (dir.exists) FileUtils.deleteDirectory(dir))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def join(from: RoleName, to: RoleName): Unit = {
|
|
|
|
|
runOn(from) {
|
|
|
|
|
Cluster(system) join node(to).address
|
|
|
|
|
createCoordinator()
|
|
|
|
|
}
|
|
|
|
|
enterBarrier(from.name + "-joined")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def createCoordinator(): Unit = {
|
2015-08-20 13:24:39 +03:00
|
|
|
val replicator = system.actorOf(Replicator.props(
|
|
|
|
|
ReplicatorSettings(system).withGossipInterval(1.second).withMaxDeltaElements(10)), "replicator")
|
|
|
|
|
|
2015-06-08 21:51:33 +02:00
|
|
|
def coordinatorProps(typeName: String, rebalanceEnabled: Boolean) = {
|
2015-06-07 14:49:38 +02:00
|
|
|
val allocationStrategy = new ShardCoordinator.LeastShardAllocationStrategy(rebalanceThreshold = 2, maxSimultaneousRebalance = 1)
|
|
|
|
|
val cfg = ConfigFactory.parseString(s"""
|
|
|
|
|
handoff-timeout = 10s
|
|
|
|
|
shard-start-timeout = 10s
|
|
|
|
|
rebalance-interval = ${if (rebalanceEnabled) "2s" else "3600s"}
|
|
|
|
|
""").withFallback(system.settings.config.getConfig("akka.cluster.sharding"))
|
|
|
|
|
val settings = ClusterShardingSettings(cfg)
|
2015-08-20 13:24:39 +03:00
|
|
|
if (settings.stateStoreMode == "persistence")
|
|
|
|
|
ShardCoordinator.props(typeName, settings, allocationStrategy)
|
|
|
|
|
else
|
|
|
|
|
ShardCoordinator.props(typeName, settings, allocationStrategy, replicator)
|
2015-06-07 14:49:38 +02:00
|
|
|
}
|
2014-11-03 08:09:32 +01:00
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
List("counter", "rebalancingCounter", "PersistentCounterEntities", "AnotherPersistentCounter",
|
2015-06-08 21:51:33 +02:00
|
|
|
"PersistentCounter", "RebalancingPersistentCounter", "AutoMigrateRegionTest").foreach { typeName ⇒
|
|
|
|
|
val rebalanceEnabled = typeName.toLowerCase.startsWith("rebalancing")
|
2015-06-01 19:03:00 +02:00
|
|
|
val singletonProps = BackoffSupervisor.props(
|
|
|
|
|
childProps = coordinatorProps(typeName, rebalanceEnabled),
|
|
|
|
|
childName = "coordinator",
|
|
|
|
|
minBackoff = 5.seconds,
|
|
|
|
|
maxBackoff = 5.seconds,
|
|
|
|
|
randomFactor = 0.1).withDeploy(Deploy.local)
|
2014-11-03 08:09:32 +01:00
|
|
|
system.actorOf(ClusterSingletonManager.props(
|
2015-06-01 19:03:00 +02:00
|
|
|
singletonProps,
|
2014-11-03 08:09:32 +01:00
|
|
|
terminationMessage = PoisonPill,
|
2015-04-29 18:23:45 +02:00
|
|
|
settings = ClusterSingletonManagerSettings(system)),
|
2015-06-08 21:51:33 +02:00
|
|
|
name = typeName + "Coordinator")
|
2014-11-03 08:09:32 +01:00
|
|
|
}
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
def createRegion(typeName: String, rememberEntities: Boolean): ActorRef = {
|
2015-06-07 14:49:38 +02:00
|
|
|
val cfg = ConfigFactory.parseString("""
|
|
|
|
|
retry-interval = 1s
|
|
|
|
|
shard-failure-backoff = 1s
|
2015-06-09 12:25:58 +02:00
|
|
|
entity-restart-backoff = 1s
|
2015-06-07 14:49:38 +02:00
|
|
|
buffer-size = 1000
|
|
|
|
|
""").withFallback(system.settings.config.getConfig("akka.cluster.sharding"))
|
|
|
|
|
val settings = ClusterShardingSettings(cfg)
|
2015-06-09 12:25:58 +02:00
|
|
|
.withRememberEntities(rememberEntities)
|
2015-06-07 14:49:38 +02:00
|
|
|
system.actorOf(ShardRegion.props(
|
|
|
|
|
typeName = typeName,
|
2015-06-30 11:43:37 +02:00
|
|
|
entityProps = qualifiedCounterProps(typeName),
|
2015-06-07 14:49:38 +02:00
|
|
|
settings = settings,
|
|
|
|
|
coordinatorPath = "/user/" + typeName + "Coordinator/singleton/coordinator",
|
2015-06-09 16:02:19 +02:00
|
|
|
extractEntityId = extractEntityId,
|
|
|
|
|
extractShardId = extractShardId,
|
2015-06-07 14:49:38 +02:00
|
|
|
handOffStopMessage = PoisonPill),
|
|
|
|
|
name = typeName + "Region")
|
|
|
|
|
}
|
2013-11-19 15:53:40 +01:00
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
lazy val region = createRegion("counter", rememberEntities = false)
|
|
|
|
|
lazy val rebalancingRegion = createRegion("rebalancingCounter", rememberEntities = false)
|
2014-08-21 16:39:24 +01:00
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
lazy val persistentEntitiesRegion = createRegion("PersistentCounterEntities", rememberEntities = true)
|
|
|
|
|
lazy val anotherPersistentRegion = createRegion("AnotherPersistentCounter", rememberEntities = true)
|
|
|
|
|
lazy val persistentRegion = createRegion("PersistentCounter", rememberEntities = true)
|
|
|
|
|
lazy val rebalancingPersistentRegion = createRegion("RebalancingPersistentCounter", rememberEntities = true)
|
|
|
|
|
lazy val autoMigrateRegion = createRegion("AutoMigrateRegionTest", rememberEntities = true)
|
2014-07-08 17:51:18 +01:00
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
s"Cluster sharding ($mode)" must {
|
2013-11-19 15:53:40 +01:00
|
|
|
|
|
|
|
|
"setup shared journal" in {
|
|
|
|
|
// start the Persistence extension
|
|
|
|
|
Persistence(system)
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
system.actorOf(Props[SharedLeveldbStore], "store")
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("peristence-started")
|
|
|
|
|
|
|
|
|
|
runOn(first, second, third, fourth, fifth, sixth) {
|
|
|
|
|
system.actorSelection(node(controller) / "user" / "store") ! Identify(None)
|
|
|
|
|
val sharedStore = expectMsgType[ActorIdentity].ref.get
|
|
|
|
|
SharedLeveldbJournal.setStore(sharedStore, system)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-1")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"work in single node cluster" in within(20 seconds) {
|
|
|
|
|
join(first, first)
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(1, Increment)
|
|
|
|
|
region ! EntityEnvelope(1, Increment)
|
|
|
|
|
region ! EntityEnvelope(1, Increment)
|
|
|
|
|
region ! EntityEnvelope(1, Decrement)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(1)
|
|
|
|
|
expectMsg(2)
|
2015-06-08 11:06:16 +02:00
|
|
|
|
|
|
|
|
region ! GetCurrentRegions
|
|
|
|
|
expectMsg(CurrentRegions(Set(Cluster(system).selfAddress)))
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-2")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"use second node" in within(20 seconds) {
|
|
|
|
|
join(second, first)
|
|
|
|
|
|
|
|
|
|
runOn(second) {
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(2, Increment)
|
|
|
|
|
region ! EntityEnvelope(2, Increment)
|
|
|
|
|
region ! EntityEnvelope(2, Increment)
|
|
|
|
|
region ! EntityEnvelope(2, Decrement)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(2)
|
|
|
|
|
expectMsg(2)
|
2014-02-19 08:06:32 +01:00
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(11, Increment)
|
|
|
|
|
region ! EntityEnvelope(12, Increment)
|
2014-02-19 08:06:32 +01:00
|
|
|
region ! Get(11)
|
|
|
|
|
expectMsg(1)
|
|
|
|
|
region ! Get(12)
|
|
|
|
|
expectMsg(1)
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("second-update")
|
|
|
|
|
runOn(first) {
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(2, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(2)
|
|
|
|
|
expectMsg(3)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(node(second) / "user" / "counterRegion" / "2" / "2")
|
2014-02-19 08:06:32 +01:00
|
|
|
|
|
|
|
|
region ! Get(11)
|
|
|
|
|
expectMsg(1)
|
|
|
|
|
// local on first
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(region.path / "11" / "11")
|
2014-02-19 08:06:32 +01:00
|
|
|
region ! Get(12)
|
|
|
|
|
expectMsg(1)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(node(second) / "user" / "counterRegion" / "0" / "12")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("first-update")
|
|
|
|
|
|
|
|
|
|
runOn(second) {
|
|
|
|
|
region ! Get(2)
|
|
|
|
|
expectMsg(3)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(region.path / "2" / "2")
|
2015-06-08 11:06:16 +02:00
|
|
|
|
|
|
|
|
region ! GetCurrentRegions
|
|
|
|
|
expectMsg(CurrentRegions(Set(Cluster(system).selfAddress, node(first).address)))
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-3")
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
"support passivation and activation of entities" in {
|
2013-11-19 15:53:40 +01:00
|
|
|
runOn(second) {
|
|
|
|
|
region ! Get(2)
|
|
|
|
|
expectMsg(3)
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(2, ReceiveTimeout)
|
2013-11-19 15:53:40 +01:00
|
|
|
// let the Passivate-Stop roundtrip begin to trigger buffering of subsequent messages
|
|
|
|
|
Thread.sleep(200)
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(2, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(2)
|
|
|
|
|
expectMsg(4)
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("after-4")
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
"support proxy only mode" in within(10.seconds) {
|
|
|
|
|
runOn(second) {
|
2015-06-07 14:49:38 +02:00
|
|
|
val cfg = ConfigFactory.parseString("""
|
|
|
|
|
retry-interval = 1s
|
|
|
|
|
buffer-size = 1000
|
|
|
|
|
""").withFallback(system.settings.config.getConfig("akka.cluster.sharding"))
|
|
|
|
|
val settings = ClusterShardingSettings(cfg)
|
2015-03-25 13:09:04 +01:00
|
|
|
val proxy = system.actorOf(ShardRegion.proxyProps(
|
|
|
|
|
typeName = "counter",
|
2015-06-07 14:49:38 +02:00
|
|
|
settings,
|
2015-03-25 13:09:04 +01:00
|
|
|
coordinatorPath = "/user/counterCoordinator/singleton/coordinator",
|
2015-06-09 16:02:19 +02:00
|
|
|
extractEntityId = extractEntityId,
|
|
|
|
|
extractShardId = extractShardId),
|
2015-03-25 13:09:04 +01:00
|
|
|
name = "regionProxy")
|
|
|
|
|
|
|
|
|
|
proxy ! Get(1)
|
|
|
|
|
expectMsg(2)
|
|
|
|
|
proxy ! Get(2)
|
|
|
|
|
expectMsg(4)
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("after-5")
|
|
|
|
|
}
|
|
|
|
|
|
2013-11-19 15:53:40 +01:00
|
|
|
"failover shards on crashed node" in within(30 seconds) {
|
|
|
|
|
// mute logging of deadLetters during shutdown of systems
|
|
|
|
|
if (!log.isDebugEnabled)
|
|
|
|
|
system.eventStream.publish(Mute(DeadLettersFilter[Any]))
|
|
|
|
|
enterBarrier("logs-muted")
|
|
|
|
|
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
testConductor.exit(second, 0).await
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("crash-second")
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
2014-02-19 08:06:32 +01:00
|
|
|
val probe1 = TestProbe()
|
|
|
|
|
awaitAssert {
|
|
|
|
|
within(1.second) {
|
|
|
|
|
region.tell(Get(2), probe1.ref)
|
|
|
|
|
probe1.expectMsg(4)
|
2015-01-16 11:09:59 +01:00
|
|
|
probe1.lastSender.path should ===(region.path / "2" / "2")
|
2014-02-19 08:06:32 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
val probe2 = TestProbe()
|
2013-11-19 15:53:40 +01:00
|
|
|
awaitAssert {
|
|
|
|
|
within(1.second) {
|
2014-02-19 08:06:32 +01:00
|
|
|
region.tell(Get(12), probe2.ref)
|
|
|
|
|
probe2.expectMsg(1)
|
2015-01-16 11:09:59 +01:00
|
|
|
probe2.lastSender.path should ===(region.path / "0" / "12")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-6")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"use third and fourth node" in within(15 seconds) {
|
|
|
|
|
join(third, first)
|
|
|
|
|
|
|
|
|
|
runOn(third) {
|
|
|
|
|
for (_ ← 1 to 10)
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(3, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(3)
|
|
|
|
|
expectMsg(10)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(region.path / "3" / "3") // local
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("third-update")
|
|
|
|
|
|
2015-08-25 19:16:46 +02:00
|
|
|
join(fourth, first)
|
|
|
|
|
|
2013-11-19 15:53:40 +01:00
|
|
|
runOn(fourth) {
|
|
|
|
|
for (_ ← 1 to 20)
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(4, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(4)
|
|
|
|
|
expectMsg(20)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(region.path / "4" / "4") // local
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("fourth-update")
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(3, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(3)
|
|
|
|
|
expectMsg(11)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(node(third) / "user" / "counterRegion" / "3" / "3")
|
2013-11-19 15:53:40 +01:00
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
region ! EntityEnvelope(4, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
region ! Get(4)
|
|
|
|
|
expectMsg(21)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(node(fourth) / "user" / "counterRegion" / "4" / "4")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("first-update")
|
|
|
|
|
|
|
|
|
|
runOn(third) {
|
|
|
|
|
region ! Get(3)
|
|
|
|
|
expectMsg(11)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(region.path / "3" / "3")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
runOn(fourth) {
|
|
|
|
|
region ! Get(4)
|
|
|
|
|
expectMsg(21)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(region.path / "4" / "4")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-7")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"recover coordinator state after coordinator crash" in within(60 seconds) {
|
|
|
|
|
join(fifth, fourth)
|
|
|
|
|
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
testConductor.exit(first, 0).await
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("crash-first")
|
|
|
|
|
|
|
|
|
|
runOn(fifth) {
|
|
|
|
|
val probe3 = TestProbe()
|
|
|
|
|
awaitAssert {
|
|
|
|
|
within(1.second) {
|
|
|
|
|
region.tell(Get(3), probe3.ref)
|
|
|
|
|
probe3.expectMsg(11)
|
2015-01-16 11:09:59 +01:00
|
|
|
probe3.lastSender.path should ===(node(third) / "user" / "counterRegion" / "3" / "3")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
val probe4 = TestProbe()
|
|
|
|
|
awaitAssert {
|
|
|
|
|
within(1.second) {
|
|
|
|
|
region.tell(Get(4), probe4.ref)
|
|
|
|
|
probe4.expectMsg(21)
|
2015-01-16 11:09:59 +01:00
|
|
|
probe4.lastSender.path should ===(node(fourth) / "user" / "counterRegion" / "4" / "4")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-8")
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|
2013-12-12 13:05:59 +01:00
|
|
|
"rebalance to nodes with less shards" in within(60 seconds) {
|
2013-11-19 15:53:40 +01:00
|
|
|
|
|
|
|
|
runOn(fourth) {
|
2014-11-03 08:09:32 +01:00
|
|
|
for (n ← 1 to 10) {
|
2015-06-09 12:25:58 +02:00
|
|
|
rebalancingRegion ! EntityEnvelope(n, Increment)
|
2014-11-03 08:09:32 +01:00
|
|
|
rebalancingRegion ! Get(n)
|
2013-11-19 15:53:40 +01:00
|
|
|
expectMsg(1)
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-11-03 08:09:32 +01:00
|
|
|
enterBarrier("rebalancing-shards-allocated")
|
2013-11-19 15:53:40 +01:00
|
|
|
|
|
|
|
|
join(sixth, third)
|
|
|
|
|
|
|
|
|
|
runOn(sixth) {
|
|
|
|
|
awaitAssert {
|
|
|
|
|
val probe = TestProbe()
|
|
|
|
|
within(3.seconds) {
|
|
|
|
|
var count = 0
|
|
|
|
|
for (n ← 1 to 10) {
|
2014-11-03 08:09:32 +01:00
|
|
|
rebalancingRegion.tell(Get(n), probe.ref)
|
2013-11-19 15:53:40 +01:00
|
|
|
probe.expectMsgType[Int]
|
2014-11-03 08:09:32 +01:00
|
|
|
if (probe.lastSender.path == rebalancingRegion.path / (n % 12).toString / n.toString)
|
2013-11-19 15:53:40 +01:00
|
|
|
count += 1
|
|
|
|
|
}
|
2014-08-25 10:26:28 +02:00
|
|
|
count should be >= (2)
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-9")
|
2013-11-19 15:53:40 +01:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"easy to use with extensions" in within(50.seconds) {
|
|
|
|
|
runOn(third, fourth, fifth, sixth) {
|
|
|
|
|
//#counter-start
|
2014-05-10 15:18:47 +02:00
|
|
|
val counterRegion: ActorRef = ClusterSharding(system).start(
|
2013-11-19 15:53:40 +01:00
|
|
|
typeName = "Counter",
|
2015-06-09 12:25:58 +02:00
|
|
|
entityProps = Props[Counter],
|
2015-06-07 14:49:38 +02:00
|
|
|
settings = ClusterShardingSettings(system),
|
2015-06-09 16:02:19 +02:00
|
|
|
extractEntityId = extractEntityId,
|
|
|
|
|
extractShardId = extractShardId)
|
2014-07-08 17:51:18 +01:00
|
|
|
//#counter-start
|
2013-11-19 15:53:40 +01:00
|
|
|
ClusterSharding(system).start(
|
|
|
|
|
typeName = "AnotherCounter",
|
2015-06-30 11:43:37 +02:00
|
|
|
entityProps = Props[AnotherCounter],
|
2015-06-07 14:49:38 +02:00
|
|
|
settings = ClusterShardingSettings(system),
|
2015-06-09 16:02:19 +02:00
|
|
|
extractEntityId = extractEntityId,
|
|
|
|
|
extractShardId = extractShardId)
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("extension-started")
|
|
|
|
|
runOn(fifth) {
|
|
|
|
|
//#counter-usage
|
|
|
|
|
val counterRegion: ActorRef = ClusterSharding(system).shardRegion("Counter")
|
2014-10-27 10:52:34 +01:00
|
|
|
counterRegion ! Get(123)
|
2013-11-19 15:53:40 +01:00
|
|
|
expectMsg(0)
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
counterRegion ! EntityEnvelope(123, Increment)
|
2014-10-27 10:52:34 +01:00
|
|
|
counterRegion ! Get(123)
|
2013-11-19 15:53:40 +01:00
|
|
|
expectMsg(1)
|
|
|
|
|
//#counter-usage
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
ClusterSharding(system).shardRegion("AnotherCounter") ! EntityEnvelope(123, Decrement)
|
2014-10-27 10:52:34 +01:00
|
|
|
ClusterSharding(system).shardRegion("AnotherCounter") ! Get(123)
|
2013-11-19 15:53:40 +01:00
|
|
|
expectMsg(-1)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("extension-used")
|
|
|
|
|
|
|
|
|
|
// sixth is a frontend node, i.e. proxy only
|
|
|
|
|
runOn(sixth) {
|
|
|
|
|
for (n ← 1000 to 1010) {
|
2015-06-09 12:25:58 +02:00
|
|
|
ClusterSharding(system).shardRegion("Counter") ! EntityEnvelope(n, Increment)
|
2013-11-19 15:53:40 +01:00
|
|
|
ClusterSharding(system).shardRegion("Counter") ! Get(n)
|
|
|
|
|
expectMsg(1)
|
2013-12-17 14:25:56 +01:00
|
|
|
lastSender.path.address should not be (Cluster(system).selfAddress)
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-10")
|
2014-05-10 15:18:47 +02:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
"easy API for starting" in within(50.seconds) {
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val counterRegionViaStart: ActorRef = ClusterSharding(system).start(
|
|
|
|
|
typeName = "ApiTest",
|
2015-06-09 12:25:58 +02:00
|
|
|
entityProps = Props[Counter],
|
2015-06-07 14:49:38 +02:00
|
|
|
settings = ClusterShardingSettings(system),
|
2015-06-09 16:02:19 +02:00
|
|
|
extractEntityId = extractEntityId,
|
|
|
|
|
extractShardId = extractShardId)
|
2014-05-10 15:18:47 +02:00
|
|
|
|
|
|
|
|
val counterRegionViaGet: ActorRef = ClusterSharding(system).shardRegion("ApiTest")
|
|
|
|
|
|
|
|
|
|
counterRegionViaStart should equal(counterRegionViaGet)
|
|
|
|
|
}
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-11")
|
2014-05-10 15:18:47 +02:00
|
|
|
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
"Persistent Cluster Shards" must {
|
2015-06-09 12:25:58 +02:00
|
|
|
"recover entities upon restart" in within(50.seconds) {
|
2014-07-08 17:51:18 +01:00
|
|
|
runOn(third, fourth, fifth) {
|
2015-06-09 12:25:58 +02:00
|
|
|
persistentEntitiesRegion
|
2014-08-21 16:39:24 +01:00
|
|
|
anotherPersistentRegion
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("persistent-started")
|
|
|
|
|
|
|
|
|
|
runOn(third) {
|
|
|
|
|
//Create an increment counter 1
|
2015-06-09 12:25:58 +02:00
|
|
|
persistentEntitiesRegion ! EntityEnvelope(1, Increment)
|
|
|
|
|
persistentEntitiesRegion ! Get(1)
|
2014-07-08 17:51:18 +01:00
|
|
|
expectMsg(1)
|
|
|
|
|
|
|
|
|
|
//Shut down the shard and confirm it's dead
|
|
|
|
|
val shard = system.actorSelection(lastSender.path.parent)
|
|
|
|
|
val region = system.actorSelection(lastSender.path.parent.parent)
|
|
|
|
|
|
|
|
|
|
//Stop the shard cleanly
|
|
|
|
|
region ! HandOff("1")
|
|
|
|
|
expectMsg(10 seconds, "ShardStopped not received", ShardStopped("1"))
|
|
|
|
|
|
2014-08-25 10:26:28 +02:00
|
|
|
val probe = TestProbe()
|
2014-07-08 17:51:18 +01:00
|
|
|
awaitAssert({
|
2014-08-25 10:26:28 +02:00
|
|
|
shard.tell(Identify(1), probe.ref)
|
|
|
|
|
probe.expectMsg(1 second, "Shard was still around", ActorIdentity(1, None))
|
2014-07-08 17:51:18 +01:00
|
|
|
}, 5 seconds, 500 millis)
|
|
|
|
|
|
|
|
|
|
//Get the path to where the shard now resides
|
2015-06-09 12:25:58 +02:00
|
|
|
persistentEntitiesRegion ! Get(13)
|
2014-07-08 17:51:18 +01:00
|
|
|
expectMsg(0)
|
|
|
|
|
|
|
|
|
|
//Check that counter 1 is now alive again, even though we have
|
|
|
|
|
// not sent a message to it via the ShardRegion
|
|
|
|
|
val counter1 = system.actorSelection(lastSender.path.parent / "1")
|
|
|
|
|
counter1 ! Identify(2)
|
2014-08-25 10:26:28 +02:00
|
|
|
expectMsgType[ActorIdentity](3 seconds).ref should not be (None)
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
counter1 ! Get(1)
|
|
|
|
|
expectMsg(1)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-shard-restart")
|
|
|
|
|
|
|
|
|
|
runOn(fourth) {
|
|
|
|
|
//Check a second region does not share the same persistent shards
|
|
|
|
|
|
|
|
|
|
//Create a separate 13 counter
|
2015-06-09 12:25:58 +02:00
|
|
|
anotherPersistentRegion ! EntityEnvelope(13, Increment)
|
2014-08-21 16:39:24 +01:00
|
|
|
anotherPersistentRegion ! Get(13)
|
2014-07-08 17:51:18 +01:00
|
|
|
expectMsg(1)
|
|
|
|
|
|
|
|
|
|
//Check that no counter "1" exists in this shard
|
|
|
|
|
val secondCounter1 = system.actorSelection(lastSender.path.parent / "1")
|
|
|
|
|
secondCounter1 ! Identify(3)
|
2014-08-25 10:26:28 +02:00
|
|
|
expectMsg(3 seconds, ActorIdentity(3, None))
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
}
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-12")
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
"permanently stop entities which passivate" in within(15.seconds) {
|
2014-07-08 17:51:18 +01:00
|
|
|
runOn(third, fourth, fifth) {
|
|
|
|
|
persistentRegion
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("cluster-started-12")
|
|
|
|
|
|
|
|
|
|
runOn(third) {
|
|
|
|
|
//Create and increment counter 1
|
2015-06-09 12:25:58 +02:00
|
|
|
persistentRegion ! EntityEnvelope(1, Increment)
|
2014-07-08 17:51:18 +01:00
|
|
|
persistentRegion ! Get(1)
|
|
|
|
|
expectMsg(1)
|
|
|
|
|
|
|
|
|
|
val counter1 = lastSender
|
|
|
|
|
val shard = system.actorSelection(counter1.path.parent)
|
|
|
|
|
val region = system.actorSelection(counter1.path.parent.parent)
|
|
|
|
|
|
|
|
|
|
//Create and increment counter 13
|
2015-06-09 12:25:58 +02:00
|
|
|
persistentRegion ! EntityEnvelope(13, Increment)
|
2014-07-08 17:51:18 +01:00
|
|
|
persistentRegion ! Get(13)
|
|
|
|
|
expectMsg(1)
|
|
|
|
|
|
|
|
|
|
val counter13 = lastSender
|
|
|
|
|
|
2015-01-16 11:09:59 +01:00
|
|
|
counter1.path.parent should ===(counter13.path.parent)
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
//Send the shard the passivate message from the counter
|
2014-08-21 16:39:24 +01:00
|
|
|
watch(counter1)
|
2014-07-08 17:51:18 +01:00
|
|
|
shard.tell(Passivate(Stop), counter1)
|
2014-08-21 16:39:24 +01:00
|
|
|
|
|
|
|
|
//Watch for the terminated message
|
|
|
|
|
expectTerminated(counter1, 5 seconds)
|
|
|
|
|
|
2014-08-25 10:26:28 +02:00
|
|
|
val probe1 = TestProbe()
|
2014-07-08 17:51:18 +01:00
|
|
|
awaitAssert({
|
|
|
|
|
//Check counter 1 is dead
|
2014-08-25 10:26:28 +02:00
|
|
|
counter1.tell(Identify(1), probe1.ref)
|
2015-06-09 12:25:58 +02:00
|
|
|
probe1.expectMsg(1 second, "Entity 1 was still around", ActorIdentity(1, None))
|
2014-07-08 17:51:18 +01:00
|
|
|
}, 5 second, 500 millis)
|
|
|
|
|
|
|
|
|
|
//Stop the shard cleanly
|
|
|
|
|
region ! HandOff("1")
|
|
|
|
|
expectMsg(10 seconds, "ShardStopped not received", ShardStopped("1"))
|
2014-08-25 10:26:28 +02:00
|
|
|
|
|
|
|
|
val probe2 = TestProbe()
|
2014-07-08 17:51:18 +01:00
|
|
|
awaitAssert({
|
2014-08-25 10:26:28 +02:00
|
|
|
shard.tell(Identify(2), probe2.ref)
|
|
|
|
|
probe2.expectMsg(1 second, "Shard was still around", ActorIdentity(2, None))
|
2014-07-08 17:51:18 +01:00
|
|
|
}, 5 seconds, 500 millis)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("shard-shutdown-12")
|
|
|
|
|
|
|
|
|
|
runOn(fourth) {
|
|
|
|
|
//Force the shard back up
|
|
|
|
|
persistentRegion ! Get(25)
|
|
|
|
|
expectMsg(0)
|
|
|
|
|
|
|
|
|
|
val shard = lastSender.path.parent
|
|
|
|
|
|
|
|
|
|
//Check counter 1 is still dead
|
|
|
|
|
system.actorSelection(shard / "1") ! Identify(3)
|
2014-08-25 10:26:28 +02:00
|
|
|
expectMsg(ActorIdentity(3, None))
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
//Check counter 13 is alive again 8
|
|
|
|
|
system.actorSelection(shard / "13") ! Identify(4)
|
2014-08-25 10:26:28 +02:00
|
|
|
expectMsgType[ActorIdentity](3 seconds).ref should not be (None)
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-13")
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
"restart entities which stop without passivating" in within(50.seconds) {
|
2014-07-08 17:51:18 +01:00
|
|
|
runOn(third, fourth) {
|
|
|
|
|
persistentRegion
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("cluster-started-12")
|
|
|
|
|
|
|
|
|
|
runOn(third) {
|
|
|
|
|
//Create and increment counter 1
|
2015-06-09 12:25:58 +02:00
|
|
|
persistentRegion ! EntityEnvelope(1, Increment)
|
2014-07-08 17:51:18 +01:00
|
|
|
persistentRegion ! Get(1)
|
|
|
|
|
expectMsg(2)
|
|
|
|
|
|
|
|
|
|
val counter1 = system.actorSelection(lastSender.path)
|
|
|
|
|
|
|
|
|
|
counter1 ! Stop
|
|
|
|
|
|
2014-08-25 10:26:28 +02:00
|
|
|
val probe = TestProbe()
|
2014-07-08 17:51:18 +01:00
|
|
|
awaitAssert({
|
2014-08-25 10:26:28 +02:00
|
|
|
counter1.tell(Identify(1), probe.ref)
|
|
|
|
|
probe.expectMsgType[ActorIdentity](1 second).ref should not be (None)
|
2014-07-08 17:51:18 +01:00
|
|
|
}, 5.seconds, 500.millis)
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-14")
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
|
2014-08-21 16:39:24 +01:00
|
|
|
"be migrated to new regions upon region failure" in within(15.seconds) {
|
2014-07-08 17:51:18 +01:00
|
|
|
|
2015-06-09 12:25:58 +02:00
|
|
|
//Start only one region, and force an entity onto that region
|
2014-07-08 17:51:18 +01:00
|
|
|
runOn(third) {
|
2015-06-09 12:25:58 +02:00
|
|
|
autoMigrateRegion ! EntityEnvelope(1, Increment)
|
2014-11-03 08:09:32 +01:00
|
|
|
autoMigrateRegion ! Get(1)
|
|
|
|
|
expectMsg(1)
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("shard1-region3")
|
|
|
|
|
|
|
|
|
|
//Start another region and test it talks to node 3
|
|
|
|
|
runOn(fourth) {
|
2015-06-09 12:25:58 +02:00
|
|
|
autoMigrateRegion ! EntityEnvelope(1, Increment)
|
2014-07-08 17:51:18 +01:00
|
|
|
|
2014-08-21 16:39:24 +01:00
|
|
|
autoMigrateRegion ! Get(1)
|
2014-07-08 17:51:18 +01:00
|
|
|
expectMsg(2)
|
2015-01-16 11:09:59 +01:00
|
|
|
lastSender.path should ===(node(third) / "user" / "AutoMigrateRegionTestRegion" / "1" / "1")
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
//Kill region 3
|
|
|
|
|
system.actorSelection(lastSender.path.parent.parent) ! PoisonPill
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("region4-up")
|
|
|
|
|
|
2014-08-25 10:26:28 +02:00
|
|
|
// Wait for migration to happen
|
2014-07-08 17:51:18 +01:00
|
|
|
//Test the shard, thus counter was moved onto node 4 and started.
|
|
|
|
|
runOn(fourth) {
|
2014-08-21 16:39:24 +01:00
|
|
|
val counter1 = system.actorSelection(system / "AutoMigrateRegionTestRegion" / "1" / "1")
|
2014-08-25 10:26:28 +02:00
|
|
|
val probe = TestProbe()
|
|
|
|
|
awaitAssert({
|
|
|
|
|
counter1.tell(Identify(1), probe.ref)
|
|
|
|
|
probe.expectMsgType[ActorIdentity](1 second).ref should not be (None)
|
|
|
|
|
}, 5.seconds, 500 millis)
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
counter1 ! Get(1)
|
|
|
|
|
expectMsg(2)
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-15")
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"ensure rebalance restarts shards" in within(50.seconds) {
|
|
|
|
|
runOn(fourth) {
|
|
|
|
|
for (i ← 2 to 12) {
|
2015-06-09 12:25:58 +02:00
|
|
|
rebalancingPersistentRegion ! EntityEnvelope(i, Increment)
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
for (i ← 2 to 12) {
|
2014-11-03 08:09:32 +01:00
|
|
|
rebalancingPersistentRegion ! Get(i)
|
2014-07-08 17:51:18 +01:00
|
|
|
expectMsg(1)
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-06-09 12:25:58 +02:00
|
|
|
enterBarrier("entities-started")
|
2014-07-08 17:51:18 +01:00
|
|
|
|
|
|
|
|
runOn(fifth) {
|
2014-11-03 08:09:32 +01:00
|
|
|
rebalancingPersistentRegion
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("fifth-joined-shard")
|
|
|
|
|
|
|
|
|
|
runOn(fifth) {
|
2014-11-03 08:09:32 +01:00
|
|
|
awaitAssert {
|
|
|
|
|
var count = 0
|
|
|
|
|
for (n ← 2 to 12) {
|
2015-06-09 12:25:58 +02:00
|
|
|
val entity = system.actorSelection(rebalancingPersistentRegion.path / (n % 12).toString / n.toString)
|
|
|
|
|
entity ! Identify(n)
|
2014-11-03 08:09:32 +01:00
|
|
|
receiveOne(3 seconds) match {
|
|
|
|
|
case ActorIdentity(id, Some(_)) if id == n ⇒ count = count + 1
|
|
|
|
|
case ActorIdentity(id, None) ⇒ //Not on the fifth shard
|
|
|
|
|
}
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
2014-11-03 08:09:32 +01:00
|
|
|
count should be >= (2)
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-25 13:09:04 +01:00
|
|
|
enterBarrier("after-16")
|
2014-07-08 17:51:18 +01:00
|
|
|
}
|
|
|
|
|
}
|
2013-11-19 15:53:40 +01:00
|
|
|
}
|
|
|
|
|
|