Automatic passivation for typed sharding, #25512

This commit is contained in:
Johan Andrén 2018-11-06 19:15:23 +01:00 committed by Patrik Nordwall
parent 3ccc67b3d6
commit 133c41375f
8 changed files with 236 additions and 18 deletions

View file

@ -4,8 +4,7 @@
package akka.cluster.sharding.typed package akka.cluster.sharding.typed
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.{ Duration, FiniteDuration }
import akka.actor.NoSerializationVerificationNeeded import akka.actor.NoSerializationVerificationNeeded
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi import akka.annotation.InternalApi
@ -42,6 +41,7 @@ object ClusterShardingSettings {
rememberEntities = untypedSettings.rememberEntities, rememberEntities = untypedSettings.rememberEntities,
journalPluginId = untypedSettings.journalPluginId, journalPluginId = untypedSettings.journalPluginId,
snapshotPluginId = untypedSettings.snapshotPluginId, snapshotPluginId = untypedSettings.snapshotPluginId,
passivateIdleEntityAfter = untypedSettings.passivateIdleEntityAfter,
stateStoreMode = StateStoreMode.byName(untypedSettings.stateStoreMode), stateStoreMode = StateStoreMode.byName(untypedSettings.stateStoreMode),
new TuningParameters(untypedSettings.tuningParameters), new TuningParameters(untypedSettings.tuningParameters),
new ClusterSingletonManagerSettings( new ClusterSingletonManagerSettings(
@ -61,6 +61,7 @@ object ClusterShardingSettings {
journalPluginId = settings.journalPluginId, journalPluginId = settings.journalPluginId,
snapshotPluginId = settings.snapshotPluginId, snapshotPluginId = settings.snapshotPluginId,
stateStoreMode = settings.stateStoreMode.name, stateStoreMode = settings.stateStoreMode.name,
passivateIdleEntityAfter = settings.passivateIdleEntityAfter,
new UntypedShardingSettings.TuningParameters( new UntypedShardingSettings.TuningParameters(
bufferSize = settings.tuningParameters.bufferSize, bufferSize = settings.tuningParameters.bufferSize,
coordinatorFailureBackoff = settings.tuningParameters.coordinatorFailureBackoff, coordinatorFailureBackoff = settings.tuningParameters.coordinatorFailureBackoff,
@ -233,6 +234,10 @@ object ClusterShardingSettings {
* be used for the internal persistence of ClusterSharding. If not defined the default * be used for the internal persistence of ClusterSharding. If not defined the default
* journal plugin is used. Note that this is not related to persistence used by the entity * journal plugin is used. Note that this is not related to persistence used by the entity
* actors. * actors.
* @param passivateIdleEntityAfter Passivate entities that have not received any message in this interval.
* Note that only messages sent through sharding are counted, so direct messages
* to the `ActorRef` of the actor or messages that it sends to itself are not counted as activity.
* Use 0 to disable automatic passivation.
* @param snapshotPluginId Absolute path to the snapshot plugin configuration entity that is to * @param snapshotPluginId Absolute path to the snapshot plugin configuration entity that is to
* be used for the internal persistence of ClusterSharding. If not defined the default * be used for the internal persistence of ClusterSharding. If not defined the default
* snapshot plugin is used. Note that this is not related to persistence used by the entity * snapshot plugin is used. Note that this is not related to persistence used by the entity
@ -246,6 +251,7 @@ final class ClusterShardingSettings(
val rememberEntities: Boolean, val rememberEntities: Boolean,
val journalPluginId: String, val journalPluginId: String,
val snapshotPluginId: String, val snapshotPluginId: String,
val passivateIdleEntityAfter: FiniteDuration,
val stateStoreMode: ClusterShardingSettings.StateStoreMode, val stateStoreMode: ClusterShardingSettings.StateStoreMode,
val tuningParameters: ClusterShardingSettings.TuningParameters, val tuningParameters: ClusterShardingSettings.TuningParameters,
val coordinatorSingletonSettings: ClusterSingletonManagerSettings) extends NoSerializationVerificationNeeded { val coordinatorSingletonSettings: ClusterSingletonManagerSettings) extends NoSerializationVerificationNeeded {
@ -290,6 +296,12 @@ final class ClusterShardingSettings(
def withStateStoreMode(stateStoreMode: ClusterShardingSettings.StateStoreMode): ClusterShardingSettings = def withStateStoreMode(stateStoreMode: ClusterShardingSettings.StateStoreMode): ClusterShardingSettings =
copy(stateStoreMode = stateStoreMode) copy(stateStoreMode = stateStoreMode)
def withPassivateIdleEntitiesAfter(duration: FiniteDuration): ClusterShardingSettings =
copy(passivateIdleEntityAfter = duration)
def withPassivateIdleEntityAfter(duration: java.time.Duration): ClusterShardingSettings =
copy(passivateIdleEntityAfter = duration.asScala)
/** /**
* The `role` of the `ClusterSingletonManagerSettings` is not used. The `role` of the * The `role` of the `ClusterSingletonManagerSettings` is not used. The `role` of the
* coordinator singleton will be the same as the `role` of `ClusterShardingSettings`. * coordinator singleton will be the same as the `role` of `ClusterShardingSettings`.
@ -305,7 +317,8 @@ final class ClusterShardingSettings(
snapshotPluginId: String = snapshotPluginId, snapshotPluginId: String = snapshotPluginId,
stateStoreMode: ClusterShardingSettings.StateStoreMode = stateStoreMode, stateStoreMode: ClusterShardingSettings.StateStoreMode = stateStoreMode,
tuningParameters: ClusterShardingSettings.TuningParameters = tuningParameters, tuningParameters: ClusterShardingSettings.TuningParameters = tuningParameters,
coordinatorSingletonSettings: ClusterSingletonManagerSettings = coordinatorSingletonSettings): ClusterShardingSettings = coordinatorSingletonSettings: ClusterSingletonManagerSettings = coordinatorSingletonSettings,
passivateIdleEntityAfter: FiniteDuration = passivateIdleEntityAfter): ClusterShardingSettings =
new ClusterShardingSettings( new ClusterShardingSettings(
numberOfShards, numberOfShards,
role, role,
@ -313,6 +326,7 @@ final class ClusterShardingSettings(
rememberEntities, rememberEntities,
journalPluginId, journalPluginId,
snapshotPluginId, snapshotPluginId,
passivateIdleEntityAfter,
stateStoreMode, stateStoreMode,
tuningParameters, tuningParameters,
coordinatorSingletonSettings) coordinatorSingletonSettings)

View file

@ -23,6 +23,10 @@ akka.cluster.sharding {
# due to rebalance or crash. # due to rebalance or crash.
remember-entities = off remember-entities = off
# Set this to a time duration to have sharding passivate entities when they have not
# gotten any message in this long time. Set to 'off' to disable.
passivate-idle-entity-after = off
# If the coordinator can't store state changes it will be stopped # If the coordinator can't store state changes it will be stopped
# and started again after this duration, with an exponential back-off # and started again after this duration, with an exponential back-off
# of up to 5 times this duration. # of up to 5 times this duration.

View file

@ -12,6 +12,7 @@ import akka.annotation.InternalApi
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.cluster.Cluster import akka.cluster.Cluster
import akka.cluster.singleton.ClusterSingletonManagerSettings import akka.cluster.singleton.ClusterSingletonManagerSettings
import akka.util.JavaDurationConverters._
object ClusterShardingSettings { object ClusterShardingSettings {
@ -53,12 +54,17 @@ object ClusterShardingSettings {
val coordinatorSingletonSettings = ClusterSingletonManagerSettings(config.getConfig("coordinator-singleton")) val coordinatorSingletonSettings = ClusterSingletonManagerSettings(config.getConfig("coordinator-singleton"))
val passivateIdleAfter =
if (config.getString("passivate-idle-entity-after").toLowerCase == "off") Duration.Zero
else config.getDuration("passivate-idle-entity-after", MILLISECONDS).millis
new ClusterShardingSettings( new ClusterShardingSettings(
role = roleOption(config.getString("role")), role = roleOption(config.getString("role")),
rememberEntities = config.getBoolean("remember-entities"), rememberEntities = config.getBoolean("remember-entities"),
journalPluginId = config.getString("journal-plugin-id"), journalPluginId = config.getString("journal-plugin-id"),
snapshotPluginId = config.getString("snapshot-plugin-id"), snapshotPluginId = config.getString("snapshot-plugin-id"),
stateStoreMode = config.getString("state-store-mode"), stateStoreMode = config.getString("state-store-mode"),
passivateIdleEntityAfter = passivateIdleAfter,
tuningParameters, tuningParameters,
coordinatorSingletonSettings) coordinatorSingletonSettings)
} }
@ -175,6 +181,7 @@ object ClusterShardingSettings {
100.milliseconds, 100.milliseconds,
5) 5)
} }
} }
} }
@ -191,6 +198,10 @@ object ClusterShardingSettings {
* be used for the internal persistence of ClusterSharding. If not defined the default * be used for the internal persistence of ClusterSharding. If not defined the default
* snapshot plugin is used. Note that this is not related to persistence used by the entity * snapshot plugin is used. Note that this is not related to persistence used by the entity
* actors. * actors.
* @param passivateIdleEntityAfter Passivate entities that have not received any message in this interval.
* Note that only messages sent through sharding are counted, so direct messages
* to the `ActorRef` of the actor or messages that it sends to itself are not counted as activity.
* Use 0 to disable automatic passivation.
* @param tuningParameters additional tuning parameters, see descriptions in reference.conf * @param tuningParameters additional tuning parameters, see descriptions in reference.conf
*/ */
final class ClusterShardingSettings( final class ClusterShardingSettings(
@ -199,9 +210,22 @@ final class ClusterShardingSettings(
val journalPluginId: String, val journalPluginId: String,
val snapshotPluginId: String, val snapshotPluginId: String,
val stateStoreMode: String, val stateStoreMode: String,
val passivateIdleEntityAfter: FiniteDuration,
val tuningParameters: ClusterShardingSettings.TuningParameters, val tuningParameters: ClusterShardingSettings.TuningParameters,
val coordinatorSingletonSettings: ClusterSingletonManagerSettings) extends NoSerializationVerificationNeeded { val coordinatorSingletonSettings: ClusterSingletonManagerSettings) extends NoSerializationVerificationNeeded {
// included for binary compatibility reasons
@deprecated("Use the ClusterShardingSettings factory methods or the constructor including passivateIdleEntityAfter instead", "2.5.18")
def this(
role: Option[String],
rememberEntities: Boolean,
journalPluginId: String,
snapshotPluginId: String,
stateStoreMode: String,
tuningParameters: ClusterShardingSettings.TuningParameters,
coordinatorSingletonSettings: ClusterSingletonManagerSettings) =
this(role, rememberEntities, journalPluginId, snapshotPluginId, stateStoreMode, Duration.Zero, tuningParameters, coordinatorSingletonSettings)
import ClusterShardingSettings.{ StateStoreModePersistence, StateStoreModeDData } import ClusterShardingSettings.{ StateStoreModePersistence, StateStoreModeDData }
require( require(
stateStoreMode == StateStoreModePersistence || stateStoreMode == StateStoreModeDData, stateStoreMode == StateStoreModePersistence || stateStoreMode == StateStoreModeDData,
@ -231,6 +255,12 @@ final class ClusterShardingSettings(
def withStateStoreMode(stateStoreMode: String): ClusterShardingSettings = def withStateStoreMode(stateStoreMode: String): ClusterShardingSettings =
copy(stateStoreMode = stateStoreMode) copy(stateStoreMode = stateStoreMode)
def withPassivateIdleAfter(duration: FiniteDuration): ClusterShardingSettings =
copy(passivateIdleAfter = duration)
def withPassivateIdleAfter(duration: java.time.Duration): ClusterShardingSettings =
copy(passivateIdleAfter = duration.asScala)
/** /**
* The `role` of the `ClusterSingletonManagerSettings` is not used. The `role` of the * The `role` of the `ClusterSingletonManagerSettings` is not used. The `role` of the
* coordinator singleton will be the same as the `role` of `ClusterShardingSettings`. * coordinator singleton will be the same as the `role` of `ClusterShardingSettings`.
@ -244,14 +274,17 @@ final class ClusterShardingSettings(
journalPluginId: String = journalPluginId, journalPluginId: String = journalPluginId,
snapshotPluginId: String = snapshotPluginId, snapshotPluginId: String = snapshotPluginId,
stateStoreMode: String = stateStoreMode, stateStoreMode: String = stateStoreMode,
passivateIdleAfter: FiniteDuration = passivateIdleEntityAfter,
tuningParameters: ClusterShardingSettings.TuningParameters = tuningParameters, tuningParameters: ClusterShardingSettings.TuningParameters = tuningParameters,
coordinatorSingletonSettings: ClusterSingletonManagerSettings = coordinatorSingletonSettings): ClusterShardingSettings = coordinatorSingletonSettings: ClusterSingletonManagerSettings = coordinatorSingletonSettings): ClusterShardingSettings =
new ClusterShardingSettings( new ClusterShardingSettings(
role, role,
rememberEntities, rememberEntities,
journalPluginId, journalPluginId,
snapshotPluginId, snapshotPluginId,
stateStoreMode, stateStoreMode,
passivateIdleAfter,
tuningParameters, tuningParameters,
coordinatorSingletonSettings) coordinatorSingletonSettings)
} }

View file

@ -12,7 +12,6 @@ import akka.actor.ActorSystem
import akka.actor.Deploy import akka.actor.Deploy
import akka.actor.Props import akka.actor.Props
import akka.actor.Terminated import akka.actor.Terminated
import akka.cluster.sharding.Shard.ShardCommand
import akka.actor.Actor import akka.actor.Actor
import akka.util.MessageBufferMap import akka.util.MessageBufferMap
@ -26,6 +25,8 @@ import akka.actor.Stash
import akka.persistence._ import akka.persistence._
import akka.actor.NoSerializationVerificationNeeded import akka.actor.NoSerializationVerificationNeeded
import scala.concurrent.duration._
/** /**
* INTERNAL API * INTERNAL API
* @see [[ClusterSharding$ ClusterSharding extension]] * @see [[ClusterSharding$ ClusterSharding extension]]
@ -115,6 +116,9 @@ private[akka] object Shard {
Props(new Shard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage)) Props(new Shard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage))
.withDeploy(Deploy.local) .withDeploy(Deploy.local)
} }
private case object PassivateIdleTick extends NoSerializationVerificationNeeded
} }
/** /**
@ -144,11 +148,20 @@ private[akka] class Shard(
var state = State.Empty var state = State.Empty
var idByRef = Map.empty[ActorRef, EntityId] var idByRef = Map.empty[ActorRef, EntityId]
var refById = Map.empty[EntityId, ActorRef] var refById = Map.empty[EntityId, ActorRef]
var lastMessageTimestamp = Map.empty[EntityId, Long]
var passivating = Set.empty[ActorRef] var passivating = Set.empty[ActorRef]
val messageBuffers = new MessageBufferMap[EntityId] val messageBuffers = new MessageBufferMap[EntityId]
var handOffStopper: Option[ActorRef] = None var handOffStopper: Option[ActorRef] = None
import context.dispatcher
val passivateIdleTask = if (settings.passivateIdleEntityAfter > Duration.Zero) {
val idleInterval = settings.passivateIdleEntityAfter / 2
Some(context.system.scheduler.schedule(idleInterval, idleInterval, self, PassivateIdleTick))
} else {
None
}
initialized() initialized()
def initialized(): Unit = context.parent ! ShardInitialized(shardId) def initialized(): Unit = context.parent ! ShardInitialized(shardId)
@ -166,6 +179,7 @@ private[akka] class Shard(
case msg: ShardRegion.StartEntityAck receiveStartEntityAck(msg) case msg: ShardRegion.StartEntityAck receiveStartEntityAck(msg)
case msg: ShardRegionCommand receiveShardRegionCommand(msg) case msg: ShardRegionCommand receiveShardRegionCommand(msg)
case msg: ShardQuery receiveShardQuery(msg) case msg: ShardQuery receiveShardQuery(msg)
case PassivateIdleTick passivateIdleEntities()
case msg if extractEntityId.isDefinedAt(msg) deliverMessage(msg, sender()) case msg if extractEntityId.isDefinedAt(msg) deliverMessage(msg, sender())
} }
@ -176,6 +190,9 @@ private[akka] class Shard(
def receiveStartEntity(start: ShardRegion.StartEntity): Unit = { def receiveStartEntity(start: ShardRegion.StartEntity): Unit = {
log.debug("Got a request from [{}] to start entity [{}] in shard [{}]", sender(), start.entityId, shardId) log.debug("Got a request from [{}] to start entity [{}] in shard [{}]", sender(), start.entityId, shardId)
if (passivateIdleTask.isDefined) {
lastMessageTimestamp = lastMessageTimestamp.updated(start.entityId, System.nanoTime())
}
getEntity(start.entityId) getEntity(start.entityId)
sender() ! ShardRegion.StartEntityAck(start.entityId, shardId) sender() ! ShardRegion.StartEntityAck(start.entityId, shardId)
} }
@ -240,6 +257,9 @@ private[akka] class Shard(
val id = idByRef(ref) val id = idByRef(ref)
idByRef -= ref idByRef -= ref
refById -= id refById -= id
if (passivateIdleTask.isDefined) {
lastMessageTimestamp -= id
}
if (messageBuffers.getOrEmpty(id).nonEmpty) { if (messageBuffers.getOrEmpty(id).nonEmpty) {
log.debug("Starting entity [{}] again, there are buffered messages for it", id) log.debug("Starting entity [{}] again, there are buffered messages for it", id)
sendMsgBuffer(EntityStarted(id)) sendMsgBuffer(EntityStarted(id))
@ -253,8 +273,6 @@ private[akka] class Shard(
def passivate(entity: ActorRef, stopMessage: Any): Unit = { def passivate(entity: ActorRef, stopMessage: Any): Unit = {
idByRef.get(entity) match { idByRef.get(entity) match {
case Some(id) if (!messageBuffers.contains(id)) { case Some(id) if (!messageBuffers.contains(id)) {
log.debug("Passivating started on entity {}", id)
passivating = passivating + entity passivating = passivating + entity
messageBuffers.add(id) messageBuffers.add(id)
entity ! stopMessage entity ! stopMessage
@ -265,6 +283,17 @@ private[akka] class Shard(
} }
} }
def passivateIdleEntities(): Unit = {
val deadline = System.nanoTime() - settings.passivateIdleEntityAfter.toNanos
val refsToPassivate = lastMessageTimestamp.collect {
case (entityId, lastMessageTimestamp) if lastMessageTimestamp < deadline refById(entityId)
}
if (refsToPassivate.nonEmpty) {
log.debug("Passivating [{}] idle entities", refsToPassivate.size)
refsToPassivate.foreach(passivate(_, handOffStopMessage))
}
}
// EntityStopped handler // EntityStopped handler
def passivateCompleted(event: EntityStopped): Unit = { def passivateCompleted(event: EntityStopped): Unit = {
log.debug("Entity stopped after passivation [{}]", event.entityId) log.debug("Entity stopped after passivation [{}]", event.entityId)
@ -295,25 +324,31 @@ private[akka] class Shard(
if (id == null || id == "") { if (id == null || id == "") {
log.warning("Id must not be empty, dropping message [{}]", msg.getClass.getName) log.warning("Id must not be empty, dropping message [{}]", msg.getClass.getName)
context.system.deadLetters ! msg context.system.deadLetters ! msg
} else if (payload.isInstanceOf[ShardRegion.StartEntity]) {
// in case it was wrapped, used in Typed
receiveStartEntity(payload.asInstanceOf[ShardRegion.StartEntity])
} else { } else {
messageBuffers.contains(id) match { payload match {
case false deliverTo(id, msg, payload, snd) case start: ShardRegion.StartEntity
// in case it was wrapped, used in Typed
receiveStartEntity(start)
case _
messageBuffers.contains(id) match {
case false deliverTo(id, msg, payload, snd)
case true if messageBuffers.totalSize >= bufferSize case true if messageBuffers.totalSize >= bufferSize
log.debug("Buffer is full, dropping message for entity [{}]", id) log.debug("Buffer is full, dropping message for entity [{}]", id)
context.system.deadLetters ! msg context.system.deadLetters ! msg
case true case true
log.debug("Message for entity [{}] buffered", id) log.debug("Message for entity [{}] buffered", id)
messageBuffers.append(id, msg, snd) messageBuffers.append(id, msg, snd)
}
} }
} }
} }
def deliverTo(id: EntityId, msg: Any, payload: Msg, snd: ActorRef): Unit = { def deliverTo(id: EntityId, msg: Any, payload: Msg, snd: ActorRef): Unit = {
if (passivateIdleTask.isDefined) {
lastMessageTimestamp = lastMessageTimestamp.updated(id, System.nanoTime())
}
val name = URLEncoder.encode(id, "utf-8") val name = URLEncoder.encode(id, "utf-8")
context.child(name) match { context.child(name) match {
case Some(actor) actor.tell(payload, snd) case Some(actor) actor.tell(payload, snd)
@ -329,10 +364,17 @@ private[akka] class Shard(
val a = context.watch(context.actorOf(entityProps(id), name)) val a = context.watch(context.actorOf(entityProps(id), name))
idByRef = idByRef.updated(a, id) idByRef = idByRef.updated(a, id)
refById = refById.updated(id, a) refById = refById.updated(id, a)
if (passivateIdleTask.isDefined) {
lastMessageTimestamp += (id -> System.nanoTime())
}
state = state.copy(state.entities + id) state = state.copy(state.entities + id)
a a
} }
} }
override def postStop(): Unit = {
passivateIdleTask.foreach(_.cancel())
}
} }
private[akka] object RememberEntityStarter { private[akka] object RememberEntityStarter {
@ -425,6 +467,9 @@ private[akka] trait RememberingShard { selfType: Shard ⇒
val id = idByRef(ref) val id = idByRef(ref)
idByRef -= ref idByRef -= ref
refById -= id refById -= id
if (passivateIdleTask.isDefined) {
lastMessageTimestamp -= id
}
if (messageBuffers.getOrEmpty(id).nonEmpty) { if (messageBuffers.getOrEmpty(id).nonEmpty) {
//Note; because we're not persisting the EntityStopped, we don't need //Note; because we're not persisting the EntityStopped, we don't need
// to persist the EntityStarted either. // to persist the EntityStarted either.

View file

@ -7,7 +7,7 @@ package akka.cluster.sharding
import java.net.URLEncoder import java.net.URLEncoder
import akka.pattern.AskTimeoutException import akka.pattern.AskTimeoutException
import akka.util.{ MessageBufferMap, Timeout } import akka.util.{ MessageBufferMap, PrettyDuration, Timeout }
import akka.pattern.{ ask, pipe } import akka.pattern.{ ask, pipe }
import akka.actor._ import akka.actor._
import akka.cluster.Cluster import akka.cluster.Cluster
@ -418,6 +418,8 @@ private[akka] class ShardRegion(
// subscribe to MemberEvent, re-subscribe when restart // subscribe to MemberEvent, re-subscribe when restart
override def preStart(): Unit = { override def preStart(): Unit = {
cluster.subscribe(self, classOf[MemberEvent]) cluster.subscribe(self, classOf[MemberEvent])
if (settings.passivateIdleEntityAfter > Duration.Zero)
log.info("Idle entities will be passivated after [{}]", PrettyDuration.format(settings.passivateIdleEntityAfter))
} }
override def postStop(): Unit = { override def postStop(): Unit = {

View file

@ -0,0 +1,102 @@
/*
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.cluster.sharding
import akka.actor.{ Actor, ActorRef, Props }
import akka.cluster.Cluster
import akka.cluster.sharding.InactiveEntityPassivationSpec.Entity.GotIt
import akka.testkit.{ AkkaSpec, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
object InactiveEntityPassivationSpec {
val config = ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.cluster.sharding.passivate-idle-entity-after = 3 s
akka.actor.serialize-messages = off
""")
object Passivate
object Entity {
def props(probe: ActorRef) = Props(new Entity(probe))
case class GotIt(id: String, msg: Any, when: Long)
}
class Entity(probe: ActorRef) extends Actor {
def id = context.self.path.name
def receive = {
case Passivate
probe ! id + " passivating"
context.stop(self)
case msg probe ! GotIt(id, msg, System.nanoTime())
}
}
val extractEntityId: ShardRegion.ExtractEntityId = {
case msg: Int (msg.toString, msg)
}
val extractShardId: ShardRegion.ExtractShardId = {
case msg: Int (msg % 10).toString
}
}
class InactiveEntityPassivationSpec extends AkkaSpec(InactiveEntityPassivationSpec.config) {
import InactiveEntityPassivationSpec._
"Passivation of inactive entities" must {
"passivate entities when they haven't seen messages for the configured duration" in {
// single node cluster
Cluster(system).join(Cluster(system).selfAddress)
val probe = TestProbe()
val settings = ClusterShardingSettings(system)
val region = ClusterSharding(system).start(
"myType",
InactiveEntityPassivationSpec.Entity.props(probe.ref),
settings,
extractEntityId,
extractShardId,
ClusterSharding(system).defaultShardAllocationStrategy(settings),
Passivate
)
region ! 1
region ! 2
val responses = Set(
probe.expectMsgType[GotIt],
probe.expectMsgType[GotIt])
responses.map(_.id) should ===(Set("1", "2"))
val timeOneSawMessage = responses.find(_.id == "1").get.when
Thread.sleep(1000)
region ! 2
probe.expectMsgType[GotIt].id should ===("2")
Thread.sleep(1000)
region ! 2
probe.expectMsgType[GotIt].id should ===("2")
// make sure "1" hasn't seen a message in 3 seconds and passivates
val timeSinceOneSawAMessage = (System.nanoTime() - timeOneSawMessage).nanos
probe.expectNoMessage(3.seconds - timeSinceOneSawAMessage)
probe.expectMsg("1 passivating")
// but it can be re activated just fine:
region ! 1
region ! 2
Set(
probe.expectMsgType[GotIt],
probe.expectMsgType[GotIt]).map(_.id) should ===(Set("1", "2"))
}
}
}

View file

@ -334,6 +334,15 @@ then supposed to stop itself. Incoming messages will be buffered by the `Shard`
between reception of `Passivate` and termination of the entity. Such buffered messages between reception of `Passivate` and termination of the entity. Such buffered messages
are thereafter delivered to a new incarnation of the entity. are thereafter delivered to a new incarnation of the entity.
### Automatic Passivation
The entities can be configured to be automatically passivated if they haven't received
a message for a while using the `akka.cluster.sharding.passivate-idle-entity-after` setting,
or by explicitly setting `ClusterShardingSettings.passivateIdleEntityAfter` to a suitable
time to keep the actor alive. Note that only messages sent through sharding are counted, so direct messages
to the `ActorRef` of the actor or messages that it sends to itself are not counted as activity.
By default automatic passivation is disabled.
<a id="cluster-sharding-remembering"></a> <a id="cluster-sharding-remembering"></a>
## Remembering Entities ## Remembering Entities

View file

@ -116,3 +116,12 @@ Scala
Java Java
: @@snip [ShardingCompileOnlyTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java) { #counter-messages #counter-passivate #counter-passivate-start } : @@snip [ShardingCompileOnlyTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java) { #counter-messages #counter-passivate #counter-passivate-start }
### Automatic Passivation
The entities can be configured to be automatically passivated if they haven't received
a message for a while using the `akka.cluster.sharding.passivate-idle-entity-after` setting,
or by explicitly setting `ClusterShardingSettings.passivateIdleEntityAfter` to a suitable
time to keep the actor alive. Note that only messages sent through sharding are counted, so direct messages
to the `ActorRef` of the actor or messages that it sends to itself are not counted as activity.
By default automatic passivation is disabled.