2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2020-01-02 07:24:59 -05:00
|
|
|
* Copyright (C) 2009-2020 Lightbend Inc. <https://www.lightbend.com>
|
2015-06-22 08:09:10 +02:00
|
|
|
*/
|
2018-03-13 23:45:55 +09:00
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
package akka.cluster.sharding
|
|
|
|
|
|
|
|
|
|
import java.net.URLEncoder
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.ActorLogging
|
|
|
|
|
import akka.actor.ActorRef
|
|
|
|
|
import akka.actor.DeadLetterSuppression
|
|
|
|
|
import akka.actor.Deploy
|
|
|
|
|
import akka.actor.NoSerializationVerificationNeeded
|
|
|
|
|
import akka.actor.Props
|
|
|
|
|
import akka.actor.Stash
|
|
|
|
|
import akka.actor.Terminated
|
|
|
|
|
import akka.actor.Timers
|
2019-05-21 17:29:11 +02:00
|
|
|
import akka.annotation.InternalStableApi
|
2017-01-18 16:28:24 +01:00
|
|
|
import akka.cluster.Cluster
|
2020-04-03 14:57:49 +02:00
|
|
|
import akka.cluster.sharding.internal.EntityRecoveryStrategy
|
|
|
|
|
import akka.cluster.sharding.internal.RememberEntitiesShardStore
|
|
|
|
|
import akka.cluster.sharding.internal.RememberEntitiesShardStore.GetEntities
|
|
|
|
|
import akka.cluster.sharding.internal.RememberEntitiesShardStoreProvider
|
|
|
|
|
import akka.cluster.sharding.internal.RememberEntityStarter
|
2019-05-07 08:50:32 +02:00
|
|
|
import akka.coordination.lease.scaladsl.Lease
|
|
|
|
|
import akka.coordination.lease.scaladsl.LeaseProvider
|
|
|
|
|
import akka.pattern.pipe
|
|
|
|
|
import akka.util.MessageBufferMap
|
2020-04-03 14:57:49 +02:00
|
|
|
import akka.util.OptionVal
|
2019-03-28 13:24:46 +01:00
|
|
|
import akka.util.PrettyDuration._
|
2019-04-05 14:56:33 +02:00
|
|
|
import akka.util.unused
|
2019-03-28 13:24:46 +01:00
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
import scala.collection.immutable.Set
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
2019-03-28 13:24:46 +01:00
|
|
|
*
|
2015-06-22 08:09:10 +02:00
|
|
|
* @see [[ClusterSharding$ ClusterSharding extension]]
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object Shard {
|
|
|
|
|
import ShardRegion.EntityId
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A Shard command
|
|
|
|
|
*/
|
2020-04-03 14:57:49 +02:00
|
|
|
sealed trait RememberEntityCommand
|
2015-06-22 08:09:10 +02:00
|
|
|
|
|
|
|
|
/**
|
2017-05-22 10:08:18 +02:00
|
|
|
* When remembering entities and the entity stops without issuing a `Passivate`, we
|
2015-06-22 08:09:10 +02:00
|
|
|
* restart it after a back off using this message.
|
|
|
|
|
*/
|
2020-04-03 14:57:49 +02:00
|
|
|
final case class RestartEntity(entity: EntityId) extends RememberEntityCommand
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2016-08-01 09:46:09 +01:00
|
|
|
/**
|
|
|
|
|
* When initialising a shard with remember entities enabled the following message is used
|
|
|
|
|
* to restart batches of entity actors at a time.
|
|
|
|
|
*/
|
2020-04-03 14:57:49 +02:00
|
|
|
final case class RestartEntities(entity: Set[EntityId]) extends RememberEntityCommand
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2015-09-23 11:24:39 +02:00
|
|
|
/**
|
|
|
|
|
* A query for information about the shard
|
|
|
|
|
*/
|
|
|
|
|
sealed trait ShardQuery
|
|
|
|
|
|
|
|
|
|
@SerialVersionUID(1L) case object GetCurrentShardState extends ShardQuery
|
|
|
|
|
|
|
|
|
|
@SerialVersionUID(1L) final case class CurrentShardState(shardId: ShardRegion.ShardId, entityIds: Set[EntityId])
|
|
|
|
|
|
2019-03-19 15:12:13 +01:00
|
|
|
@SerialVersionUID(1L) case object GetShardStats extends ShardQuery with ClusterShardingSerializable
|
2015-09-23 11:24:39 +02:00
|
|
|
|
|
|
|
|
@SerialVersionUID(1L) final case class ShardStats(shardId: ShardRegion.ShardId, entityCount: Int)
|
2019-03-19 15:12:13 +01:00
|
|
|
extends ClusterShardingSerializable
|
2015-09-23 11:24:39 +02:00
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
final case class LeaseAcquireResult(acquired: Boolean, reason: Option[Throwable]) extends DeadLetterSuppression
|
|
|
|
|
final case class LeaseLost(reason: Option[Throwable]) extends DeadLetterSuppression
|
|
|
|
|
|
|
|
|
|
final case object LeaseRetry extends DeadLetterSuppression
|
|
|
|
|
private val LeaseRetryTimer = "lease-retry"
|
|
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
def props(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
entityProps: String => Props,
|
|
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
2019-04-09 16:11:35 +00:00
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
2019-03-13 10:56:20 +01:00
|
|
|
handOffStopMessage: Any,
|
2020-04-03 14:57:49 +02:00
|
|
|
rememberEntitiesProvider: Option[RememberEntitiesShardStoreProvider]): Props =
|
|
|
|
|
Props(
|
|
|
|
|
new Shard(
|
|
|
|
|
typeName,
|
|
|
|
|
shardId,
|
|
|
|
|
entityProps,
|
|
|
|
|
settings,
|
|
|
|
|
extractEntityId,
|
|
|
|
|
extractShardId,
|
|
|
|
|
handOffStopMessage,
|
|
|
|
|
rememberEntitiesProvider)).withDeploy(Deploy.local)
|
2018-11-06 19:15:23 +01:00
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
case object PassivateIdleTick extends NoSerializationVerificationNeeded
|
2018-11-06 19:15:23 +01:00
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
private final case class RememberedEntityIds(ids: Set[EntityId])
|
|
|
|
|
private final case class RememberEntityStoreCrashed(store: ActorRef)
|
|
|
|
|
private final case object AsyncWriteDone
|
|
|
|
|
|
|
|
|
|
private val RememberEntityTimeoutKey = "RememberEntityTimeout"
|
|
|
|
|
case class RememberEntityTimeout(operation: RememberEntitiesShardStore.Command)
|
|
|
|
|
|
|
|
|
|
// FIXME Leaving this on while we are working on the remember entities refactor
|
|
|
|
|
// should it go in settings perhaps, useful for tricky sharding bugs?
|
|
|
|
|
final val VerboseDebug = true
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* This actor creates children entity actors on demand that it is told to be
|
|
|
|
|
* responsible for.
|
|
|
|
|
*
|
|
|
|
|
* @see [[ClusterSharding$ ClusterSharding extension]]
|
|
|
|
|
*/
|
2020-04-03 14:57:49 +02:00
|
|
|
// FIXME I broke bin comp here
|
2019-05-29 07:09:12 +01:00
|
|
|
@InternalStableApi
|
2019-03-13 10:56:20 +01:00
|
|
|
private[akka] class Shard(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
entityProps: String => Props,
|
|
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
2019-04-09 16:11:35 +00:00
|
|
|
@unused extractShardId: ShardRegion.ExtractShardId,
|
2020-04-03 14:57:49 +02:00
|
|
|
handOffStopMessage: Any,
|
|
|
|
|
rememberEntitiesProvider: Option[RememberEntitiesShardStoreProvider])
|
2019-03-11 10:38:24 +01:00
|
|
|
extends Actor
|
2019-03-28 13:24:46 +01:00
|
|
|
with ActorLogging
|
2020-04-03 14:57:49 +02:00
|
|
|
with Stash
|
2019-03-28 13:24:46 +01:00
|
|
|
with Timers {
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
import Shard._
|
2019-05-07 08:50:32 +02:00
|
|
|
import ShardCoordinator.Internal.HandOff
|
|
|
|
|
import ShardCoordinator.Internal.ShardStopped
|
|
|
|
|
import ShardRegion.EntityId
|
|
|
|
|
import ShardRegion.Passivate
|
|
|
|
|
import ShardRegion.ShardInitialized
|
|
|
|
|
import ShardRegion.handOffStopperProps
|
2015-06-22 08:09:10 +02:00
|
|
|
import akka.cluster.sharding.ShardCoordinator.Internal.CoordinatorMessage
|
|
|
|
|
import akka.cluster.sharding.ShardRegion.ShardRegionCommand
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
private val rememberEntitiesStore: Option[ActorRef] =
|
|
|
|
|
rememberEntitiesProvider.map { provider =>
|
|
|
|
|
val store = context.actorOf(provider.shardStoreProps(shardId).withDeploy(Deploy.local), "RememberEntitiesStore")
|
|
|
|
|
context.watchWith(store, RememberEntityStoreCrashed(store))
|
|
|
|
|
store
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private val rememberedEntitiesRecoveryStrategy: EntityRecoveryStrategy = {
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
entityRecoveryStrategy match {
|
|
|
|
|
case "all" => EntityRecoveryStrategy.allStrategy()
|
|
|
|
|
case "constant" =>
|
|
|
|
|
EntityRecoveryStrategy.constantStrategy(
|
|
|
|
|
context.system,
|
|
|
|
|
entityRecoveryConstantRateStrategyFrequency,
|
|
|
|
|
entityRecoveryConstantRateStrategyNumberOfEntities)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// this will contain entity ids even if not yet started (or stopped without graceful stop)
|
|
|
|
|
private var entityIds: Set[EntityId] = Set.empty
|
|
|
|
|
|
|
|
|
|
private var idByRef = Map.empty[ActorRef, EntityId]
|
|
|
|
|
private var refById = Map.empty[EntityId, ActorRef]
|
|
|
|
|
|
|
|
|
|
private var lastMessageTimestamp = Map.empty[EntityId, Long]
|
|
|
|
|
|
|
|
|
|
// that an entity is passivating it is added to the passivating set and its id is added to the message buffers
|
|
|
|
|
// to buffer any messages coming in during passivation
|
|
|
|
|
private var passivating = Set.empty[ActorRef]
|
|
|
|
|
private val messageBuffers = new MessageBufferMap[EntityId]
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private var handOffStopper: Option[ActorRef] = None
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2018-11-06 19:15:23 +01:00
|
|
|
import context.dispatcher
|
2020-04-03 14:57:49 +02:00
|
|
|
private val passivateIdleTask = if (settings.shouldPassivateIdleEntities) {
|
2018-11-06 19:15:23 +01:00
|
|
|
val idleInterval = settings.passivateIdleEntityAfter / 2
|
2019-05-27 11:53:26 +02:00
|
|
|
Some(context.system.scheduler.scheduleWithFixedDelay(idleInterval, idleInterval, self, PassivateIdleTick))
|
2018-11-06 19:15:23 +01:00
|
|
|
} else {
|
|
|
|
|
None
|
|
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private val lease = settings.leaseSettings.map(
|
2019-03-28 13:24:46 +01:00
|
|
|
ls =>
|
|
|
|
|
LeaseProvider(context.system).getLease(
|
|
|
|
|
s"${context.system.name}-shard-$typeName-$shardId",
|
|
|
|
|
ls.leaseImplementation,
|
|
|
|
|
Cluster(context.system).selfAddress.hostPort))
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private val leaseRetryInterval = settings.leaseSettings match {
|
2019-03-28 13:24:46 +01:00
|
|
|
case Some(l) => l.leaseRetryInterval
|
|
|
|
|
case None => 5.seconds // not used
|
|
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
def receive: Receive = {
|
|
|
|
|
case _ => throw new IllegalStateException("Default receive never expected to actually be used")
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
acquireLeaseIfNeeded()
|
|
|
|
|
}
|
2015-08-20 13:24:39 +03:00
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
// ===== lease handling initialization =====
|
2019-03-28 13:24:46 +01:00
|
|
|
def acquireLeaseIfNeeded(): Unit = {
|
|
|
|
|
lease match {
|
|
|
|
|
case Some(l) =>
|
|
|
|
|
tryGetLease(l)
|
|
|
|
|
context.become(awaitingLease())
|
|
|
|
|
case None =>
|
2020-04-03 14:57:49 +02:00
|
|
|
tryLoadRememberedEntities()
|
2019-03-28 13:24:46 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Don't send back ShardInitialized so that messages are buffered in the ShardRegion
|
|
|
|
|
// while awaiting the lease
|
2019-05-07 08:50:32 +02:00
|
|
|
private def awaitingLease(): Receive = {
|
2019-03-28 13:24:46 +01:00
|
|
|
case LeaseAcquireResult(true, _) =>
|
2020-04-03 14:57:49 +02:00
|
|
|
log.debug("Lease acquired")
|
|
|
|
|
tryLoadRememberedEntities()
|
2019-03-28 13:24:46 +01:00
|
|
|
case LeaseAcquireResult(false, None) =>
|
|
|
|
|
log.error(
|
|
|
|
|
"Failed to get lease for shard type [{}] id [{}]. Retry in {}",
|
|
|
|
|
typeName,
|
|
|
|
|
shardId,
|
|
|
|
|
leaseRetryInterval.pretty)
|
|
|
|
|
timers.startSingleTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
|
|
|
|
|
case LeaseAcquireResult(false, Some(t)) =>
|
|
|
|
|
log.error(
|
|
|
|
|
t,
|
|
|
|
|
"Failed to get lease for shard type [{}] id [{}]. Retry in {}",
|
|
|
|
|
typeName,
|
|
|
|
|
shardId,
|
|
|
|
|
leaseRetryInterval)
|
|
|
|
|
timers.startSingleTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
|
|
|
|
|
case LeaseRetry =>
|
|
|
|
|
tryGetLease(lease.get)
|
|
|
|
|
case ll: LeaseLost =>
|
|
|
|
|
receiveLeaseLost(ll)
|
2020-04-03 14:57:49 +02:00
|
|
|
case _ =>
|
|
|
|
|
stash()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def tryGetLease(l: Lease): Unit = {
|
|
|
|
|
log.info("Acquiring lease {}", l.settings)
|
|
|
|
|
pipe(l.acquire(reason => self ! LeaseLost(reason)).map(r => LeaseAcquireResult(r, None)).recover {
|
|
|
|
|
case t => LeaseAcquireResult(acquired = false, Some(t))
|
|
|
|
|
}).to(self)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// ===== remember entities initialization =====
|
|
|
|
|
def tryLoadRememberedEntities(): Unit = {
|
|
|
|
|
rememberEntitiesStore match {
|
|
|
|
|
case Some(store) =>
|
|
|
|
|
log.debug("Waiting for load of entity ids using [{}] to complete", store)
|
|
|
|
|
store ! RememberEntitiesShardStore.GetEntities
|
|
|
|
|
timers.startSingleTimer(
|
|
|
|
|
RememberEntityTimeoutKey,
|
|
|
|
|
RememberEntityTimeout(GetEntities),
|
|
|
|
|
settings.tuningParameters.waitingForStateTimeout)
|
|
|
|
|
context.become(awaitingRememberedEntities())
|
|
|
|
|
case None =>
|
|
|
|
|
onEntitiesRemembered(Set.empty)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def awaitingRememberedEntities(): Receive = {
|
|
|
|
|
case RememberEntitiesShardStore.RememberedEntities(entityIds) =>
|
|
|
|
|
timers.cancel(RememberEntityTimeoutKey)
|
|
|
|
|
onEntitiesRemembered(entityIds)
|
|
|
|
|
case RememberEntityTimeout(GetEntities) =>
|
|
|
|
|
loadingEntityIdsFailed()
|
|
|
|
|
case msg =>
|
|
|
|
|
if (VerboseDebug)
|
|
|
|
|
log.debug("Got msg of type [{}] from [{}] while waiting for remember entitites", msg.getClass, sender())
|
|
|
|
|
stash()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def loadingEntityIdsFailed(): Unit = {
|
|
|
|
|
log.error(
|
|
|
|
|
"Failed to load initial entity ids from remember entities store within [{}], stopping shard for backoff and restart",
|
|
|
|
|
settings.tuningParameters.waitingForStateTimeout.pretty)
|
|
|
|
|
// parent ShardRegion supervisor will notice that it terminated and will start it again, after backoff
|
|
|
|
|
context.stop(self)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def onEntitiesRemembered(ids: Set[EntityId]): Unit = {
|
|
|
|
|
log.debug("Shard initialized")
|
|
|
|
|
if (ids.nonEmpty) {
|
|
|
|
|
entityIds = ids
|
|
|
|
|
restartRememberedEntities(ids)
|
|
|
|
|
} else {}
|
|
|
|
|
context.parent ! ShardInitialized(shardId)
|
|
|
|
|
context.become(idle)
|
|
|
|
|
unstashAll()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def restartRememberedEntities(ids: Set[EntityId]): Unit = {
|
|
|
|
|
log.debug(
|
|
|
|
|
"Shard starting [{}] remembered entities using strategy [{}]",
|
|
|
|
|
ids.size,
|
|
|
|
|
rememberedEntitiesRecoveryStrategy)
|
|
|
|
|
rememberedEntitiesRecoveryStrategy.recoverEntities(ids).foreach { scheduledRecovery =>
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
scheduledRecovery.filter(_.nonEmpty).map(RestartEntities).pipeTo(self)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def restartEntities(ids: Set[EntityId]): Unit = {
|
|
|
|
|
log.debug("Restarting set of [{}] entities", ids.size)
|
|
|
|
|
context.actorOf(RememberEntityStarter.props(context.parent, ids, settings, sender()))
|
2019-03-28 13:24:46 +01:00
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
// ===== shard up and running =====
|
|
|
|
|
|
|
|
|
|
def idle: Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Terminated(ref) => receiveTerminated(ref)
|
|
|
|
|
case msg: CoordinatorMessage => receiveCoordinatorMessage(msg)
|
2020-04-03 14:57:49 +02:00
|
|
|
case msg: RememberEntityCommand => receiveRememberEntityCommand(msg)
|
2019-02-09 15:25:39 +01:00
|
|
|
case msg: ShardRegion.StartEntity => receiveStartEntity(msg)
|
|
|
|
|
case msg: ShardRegion.StartEntityAck => receiveStartEntityAck(msg)
|
|
|
|
|
case msg: ShardRegionCommand => receiveShardRegionCommand(msg)
|
|
|
|
|
case msg: ShardQuery => receiveShardQuery(msg)
|
|
|
|
|
case PassivateIdleTick => passivateIdleEntities()
|
2019-03-28 13:24:46 +01:00
|
|
|
case msg: LeaseLost => receiveLeaseLost(msg)
|
2020-04-03 14:57:49 +02:00
|
|
|
case msg: RememberEntityStoreCrashed => rememberEntityStoreCrashed(msg)
|
|
|
|
|
case msg if extractEntityId.isDefinedAt(msg) => deliverMessage(msg, sender(), OptionVal.None)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def waitForAsyncWrite(entityId: EntityId, command: RememberEntitiesShardStore.Command)(
|
|
|
|
|
whenDone: EntityId => Unit): Unit = {
|
|
|
|
|
rememberEntitiesStore match {
|
|
|
|
|
case None =>
|
|
|
|
|
whenDone(entityId)
|
|
|
|
|
|
|
|
|
|
case Some(store) =>
|
|
|
|
|
if (VerboseDebug) log.debug("Update of [{}] [{}] triggered", entityId, command)
|
|
|
|
|
store ! command
|
|
|
|
|
timers.startSingleTimer(
|
|
|
|
|
RememberEntityTimeoutKey,
|
|
|
|
|
RememberEntityTimeout(command),
|
|
|
|
|
// FIXME this timeout needs to match the timeout used in the ddata shard write since that tries 3 times
|
|
|
|
|
// and this could always fail before ddata store completes retrying writes
|
|
|
|
|
settings.tuningParameters.updatingStateTimeout)
|
|
|
|
|
|
|
|
|
|
context.become {
|
|
|
|
|
case RememberEntitiesShardStore.UpdateDone(entityId) =>
|
|
|
|
|
if (VerboseDebug) log.debug("Update of [{}] {} done", entityId, command)
|
|
|
|
|
timers.cancel(RememberEntityTimeoutKey)
|
|
|
|
|
whenDone(entityId)
|
|
|
|
|
context.become(idle)
|
|
|
|
|
unstashAll()
|
|
|
|
|
case RememberEntityTimeout(`command`) =>
|
|
|
|
|
throw new RuntimeException(
|
|
|
|
|
s"Async write for entityId $entityId timed out after ${settings.tuningParameters.updatingStateTimeout.pretty}")
|
|
|
|
|
|
|
|
|
|
// below cases should handle same messages as in idle
|
|
|
|
|
case _: Terminated => stash()
|
|
|
|
|
case _: CoordinatorMessage => stash()
|
|
|
|
|
case _: RememberEntityCommand => stash()
|
|
|
|
|
case _: ShardRegion.StartEntity => stash()
|
|
|
|
|
case _: ShardRegion.StartEntityAck => stash()
|
|
|
|
|
case _: ShardRegionCommand => stash()
|
|
|
|
|
case msg: ShardQuery => receiveShardQuery(msg)
|
|
|
|
|
case PassivateIdleTick => stash()
|
|
|
|
|
case msg: LeaseLost => receiveLeaseLost(msg)
|
|
|
|
|
case msg: RememberEntityStoreCrashed => rememberEntityStoreCrashed(msg)
|
|
|
|
|
case msg if extractEntityId.isDefinedAt(msg) => deliverMessage(msg, sender(), OptionVal.Some(entityId))
|
|
|
|
|
case msg =>
|
|
|
|
|
// shouldn't be any other message types, but just in case
|
|
|
|
|
log.debug(
|
|
|
|
|
"Stashing unexpected message [{}] while waiting for remember entities update of {}",
|
|
|
|
|
msg.getClass,
|
|
|
|
|
entityId)
|
|
|
|
|
stash()
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
def receiveLeaseLost(msg: LeaseLost): Unit = {
|
|
|
|
|
// The shard region will re-create this when it receives a message for this shard
|
2020-04-03 14:57:49 +02:00
|
|
|
log.error(
|
|
|
|
|
"Shard type [{}] id [{}] lease lost, stopping shard and killing [{}] entities.{}",
|
|
|
|
|
typeName,
|
|
|
|
|
shardId,
|
|
|
|
|
entityIds.size,
|
|
|
|
|
msg.reason match {
|
|
|
|
|
case Some(reason) => s" Reason for losing lease: $reason"
|
|
|
|
|
case None => ""
|
|
|
|
|
})
|
2019-03-28 13:24:46 +01:00
|
|
|
// Stop entities ASAP rather than send termination message
|
|
|
|
|
context.stop(self)
|
|
|
|
|
}
|
2019-05-07 08:50:32 +02:00
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
private def receiveRememberEntityCommand(msg: RememberEntityCommand): Unit = msg match {
|
|
|
|
|
// these are only used with remembering entities upon start
|
|
|
|
|
case RestartEntity(id) =>
|
|
|
|
|
// starting because it was remembered as started on shard startup (note that a message starting
|
|
|
|
|
// it up could already have arrived and in that case it will already be started)
|
|
|
|
|
getOrCreateEntity(id)
|
2019-02-09 15:25:39 +01:00
|
|
|
case RestartEntities(ids) => restartEntities(ids)
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def receiveStartEntity(start: ShardRegion.StartEntity): Unit = {
|
2018-12-06 14:49:47 +01:00
|
|
|
val requester = sender()
|
|
|
|
|
log.debug("Got a request from [{}] to start entity [{}] in shard [{}]", requester, start.entityId, shardId)
|
2019-05-07 08:50:32 +02:00
|
|
|
touchLastMessageTimestamp(start.entityId)
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
if (entityIds(start.entityId)) {
|
2019-05-07 08:50:32 +02:00
|
|
|
getOrCreateEntity(start.entityId)
|
|
|
|
|
requester ! ShardRegion.StartEntityAck(start.entityId, shardId)
|
|
|
|
|
} else {
|
2020-04-03 14:57:49 +02:00
|
|
|
waitForAsyncWrite(start.entityId, RememberEntitiesShardStore.AddEntity(start.entityId)) { id =>
|
|
|
|
|
getOrCreateEntity(id)
|
|
|
|
|
sendMsgBuffer(id)
|
|
|
|
|
requester ! ShardRegion.StartEntityAck(id, shardId)
|
2019-05-07 08:50:32 +02:00
|
|
|
}
|
2018-11-06 19:15:23 +01:00
|
|
|
}
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def receiveStartEntityAck(ack: ShardRegion.StartEntityAck): Unit = {
|
2020-04-03 14:57:49 +02:00
|
|
|
if (ack.shardId != shardId && entityIds(ack.entityId)) {
|
2017-05-22 10:08:18 +02:00
|
|
|
log.debug("Entity [{}] previously owned by shard [{}] started in shard [{}]", ack.entityId, shardId, ack.shardId)
|
2020-04-03 14:57:49 +02:00
|
|
|
|
|
|
|
|
waitForAsyncWrite(ack.entityId, RememberEntitiesShardStore.RemoveEntity(ack.entityId)) { id =>
|
|
|
|
|
entityIds = entityIds - id
|
|
|
|
|
messageBuffers.remove(id)
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def receiveShardRegionCommand(msg: ShardRegionCommand): Unit = msg match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Passivate(stopMessage) => passivate(sender(), stopMessage)
|
|
|
|
|
case _ => unhandled(msg)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def receiveCoordinatorMessage(msg: CoordinatorMessage): Unit = msg match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case HandOff(`shardId`) => handOff(sender())
|
|
|
|
|
case HandOff(shard) => log.warning("Shard [{}] can not hand off for another Shard [{}]", shardId, shard)
|
|
|
|
|
case _ => unhandled(msg)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2015-09-23 11:24:39 +02:00
|
|
|
def receiveShardQuery(msg: ShardQuery): Unit = msg match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case GetCurrentShardState => sender() ! CurrentShardState(shardId, refById.keySet)
|
2020-04-03 14:57:49 +02:00
|
|
|
case GetShardStats => sender() ! ShardStats(shardId, entityIds.size)
|
2015-09-23 11:24:39 +02:00
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def handOff(replyTo: ActorRef): Unit = handOffStopper match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Some(_) => log.warning("HandOff shard [{}] received during existing handOff", shardId)
|
|
|
|
|
case None =>
|
2015-06-22 08:09:10 +02:00
|
|
|
log.debug("HandOff shard [{}]", shardId)
|
|
|
|
|
|
2019-10-09 10:53:50 +02:00
|
|
|
if (idByRef.nonEmpty) {
|
2018-12-04 20:50:58 +03:30
|
|
|
val entityHandOffTimeout = (settings.tuningParameters.handOffTimeout - 5.seconds).max(1.seconds)
|
2019-10-09 10:53:50 +02:00
|
|
|
log.debug("Starting HandOffStopper for shard [{}] to terminate [{}] entities.", shardId, idByRef.keySet.size)
|
2019-03-11 10:38:24 +01:00
|
|
|
handOffStopper = Some(
|
|
|
|
|
context.watch(context.actorOf(
|
|
|
|
|
handOffStopperProps(shardId, replyTo, idByRef.keySet, handOffStopMessage, entityHandOffTimeout))))
|
2015-06-22 08:09:10 +02:00
|
|
|
|
|
|
|
|
//During hand off we only care about watching for termination of the hand off stopper
|
2019-03-11 10:38:24 +01:00
|
|
|
context.become {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Terminated(ref) => receiveTerminated(ref)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
replyTo ! ShardStopped(shardId)
|
2019-03-11 10:38:24 +01:00
|
|
|
context.stop(self)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def receiveTerminated(ref: ActorRef): Unit = {
|
2017-02-21 16:17:19 +03:00
|
|
|
if (handOffStopper.contains(ref))
|
2019-03-11 10:38:24 +01:00
|
|
|
context.stop(self)
|
2015-06-22 08:54:42 +02:00
|
|
|
else if (idByRef.contains(ref) && handOffStopper.isEmpty)
|
|
|
|
|
entityTerminated(ref)
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2019-05-29 07:09:12 +01:00
|
|
|
@InternalStableApi
|
2015-06-22 08:54:42 +02:00
|
|
|
def entityTerminated(ref: ActorRef): Unit = {
|
2020-04-03 14:57:49 +02:00
|
|
|
import settings.tuningParameters._
|
2015-06-22 08:54:42 +02:00
|
|
|
val id = idByRef(ref)
|
2017-05-15 15:27:53 +02:00
|
|
|
idByRef -= ref
|
|
|
|
|
refById -= id
|
2018-11-06 19:15:23 +01:00
|
|
|
if (passivateIdleTask.isDefined) {
|
|
|
|
|
lastMessageTimestamp -= id
|
|
|
|
|
}
|
2017-03-15 13:31:47 +01:00
|
|
|
if (messageBuffers.getOrEmpty(id).nonEmpty) {
|
2020-04-03 14:57:49 +02:00
|
|
|
// Note; because we're not persisting the EntityStopped, we don't need
|
|
|
|
|
// to persist the EntityStarted either.
|
2015-06-22 08:54:42 +02:00
|
|
|
log.debug("Starting entity [{}] again, there are buffered messages for it", id)
|
2020-04-03 14:57:49 +02:00
|
|
|
sendMsgBuffer(id)
|
2015-06-22 08:54:42 +02:00
|
|
|
} else {
|
2020-04-03 14:57:49 +02:00
|
|
|
if (!passivating.contains(ref)) {
|
|
|
|
|
log.debug("Entity [{}] stopped without passivating, will restart after backoff", id)
|
|
|
|
|
// note that it's not removed from state here, will be started again via RestartEntity
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
context.system.scheduler.scheduleOnce(entityRestartBackoff, self, RestartEntity(id))
|
|
|
|
|
} else {
|
|
|
|
|
// FIXME optional wait for completion as optimization where stops are not critical
|
|
|
|
|
waitForAsyncWrite(id, RememberEntitiesShardStore.RemoveEntity(id))(passivateCompleted)
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
2015-06-22 08:54:42 +02:00
|
|
|
|
|
|
|
|
passivating = passivating - ref
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
private def passivate(entity: ActorRef, stopMessage: Any): Unit = {
|
2015-06-22 08:09:10 +02:00
|
|
|
idByRef.get(entity) match {
|
2019-03-11 10:38:24 +01:00
|
|
|
case Some(id) =>
|
|
|
|
|
if (!messageBuffers.contains(id)) {
|
2020-04-03 14:57:49 +02:00
|
|
|
if (VerboseDebug)
|
|
|
|
|
log.debug("Passivation started for {}", entity)
|
2019-03-11 10:38:24 +01:00
|
|
|
passivating = passivating + entity
|
|
|
|
|
messageBuffers.add(id)
|
|
|
|
|
entity ! stopMessage
|
|
|
|
|
} else {
|
2020-04-03 14:57:49 +02:00
|
|
|
log.debug("Passivation already in progress for {}. Not sending stopMessage back to entity", entity)
|
2019-03-11 10:38:24 +01:00
|
|
|
}
|
2020-04-03 14:57:49 +02:00
|
|
|
case None => log.debug("Unknown entity {}. Not sending stopMessage back to entity", entity)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2019-05-07 08:50:32 +02:00
|
|
|
def touchLastMessageTimestamp(id: EntityId): Unit = {
|
|
|
|
|
if (passivateIdleTask.isDefined) {
|
|
|
|
|
lastMessageTimestamp = lastMessageTimestamp.updated(id, System.nanoTime())
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def passivateIdleEntities(): Unit = {
|
2018-11-06 19:15:23 +01:00
|
|
|
val deadline = System.nanoTime() - settings.passivateIdleEntityAfter.toNanos
|
|
|
|
|
val refsToPassivate = lastMessageTimestamp.collect {
|
2019-02-09 15:25:39 +01:00
|
|
|
case (entityId, lastMessageTimestamp) if lastMessageTimestamp < deadline => refById(entityId)
|
2018-11-06 19:15:23 +01:00
|
|
|
}
|
|
|
|
|
if (refsToPassivate.nonEmpty) {
|
|
|
|
|
log.debug("Passivating [{}] idle entities", refsToPassivate.size)
|
|
|
|
|
refsToPassivate.foreach(passivate(_, handOffStopMessage))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
// After entity stopped
|
|
|
|
|
def passivateCompleted(entityId: EntityId): Unit = {
|
|
|
|
|
val hasBufferedMessages = messageBuffers.getOrEmpty(entityId).nonEmpty
|
|
|
|
|
entityIds = entityIds - entityId
|
2019-05-20 08:37:55 +02:00
|
|
|
if (hasBufferedMessages) {
|
2020-04-03 14:57:49 +02:00
|
|
|
log.debug("Entity stopped after passivation [{}], but will be started again due to buffered messages.", entityId)
|
|
|
|
|
waitForAsyncWrite(entityId, RememberEntitiesShardStore.AddEntity(entityId))(sendMsgBuffer)
|
2019-05-20 08:37:55 +02:00
|
|
|
} else {
|
2020-04-03 14:57:49 +02:00
|
|
|
log.debug("Entity stopped after passivation [{}]", entityId)
|
|
|
|
|
dropBufferFor(entityId)
|
2019-05-20 08:37:55 +02:00
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
/**
|
|
|
|
|
* @param entityIdWaitingForWrite an id for an remember entity write in progress, if non empty messages for that id
|
|
|
|
|
* will be buffered
|
|
|
|
|
*/
|
|
|
|
|
private def deliverMessage(msg: Any, snd: ActorRef, entityIdWaitingForWrite: OptionVal[EntityId]): Unit = {
|
2015-06-22 08:09:10 +02:00
|
|
|
val (id, payload) = extractEntityId(msg)
|
|
|
|
|
if (id == null || id == "") {
|
|
|
|
|
log.warning("Id must not be empty, dropping message [{}]", msg.getClass.getName)
|
|
|
|
|
context.system.deadLetters ! msg
|
|
|
|
|
} else {
|
2018-11-06 19:15:23 +01:00
|
|
|
payload match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case start: ShardRegion.StartEntity =>
|
2020-04-03 14:57:49 +02:00
|
|
|
// we can only start a new entity if we are not currently waiting for another write
|
|
|
|
|
if (entityIdWaitingForWrite.isEmpty) receiveStartEntity(start)
|
|
|
|
|
// write in progress, see waitForAsyncWrite for unstash
|
|
|
|
|
else stash()
|
2019-02-09 15:25:39 +01:00
|
|
|
case _ =>
|
2020-04-03 14:57:49 +02:00
|
|
|
if (messageBuffers.contains(id) || entityIdWaitingForWrite.contains(id)) {
|
|
|
|
|
// either:
|
|
|
|
|
// 1. entity is passivating, buffer until passivation complete (id in message buffers)
|
|
|
|
|
// 2. we are waiting for storing entity start or stop with remember entities to complete
|
|
|
|
|
// and want to buffer until write completes
|
|
|
|
|
if (VerboseDebug) {
|
|
|
|
|
if (entityIdWaitingForWrite.contains(id))
|
|
|
|
|
log.debug("Buffering message [{}] to [{}] because of write in progress for it", msg.getClass, id)
|
|
|
|
|
else
|
|
|
|
|
log.debug("Buffering message [{}] to [{}] because passivation in progress for it", msg.getClass, id)
|
|
|
|
|
}
|
2019-05-07 08:50:32 +02:00
|
|
|
appendToMessageBuffer(id, msg, snd)
|
2020-04-03 14:57:49 +02:00
|
|
|
} else {
|
|
|
|
|
// With remember entities enabled we may be in the process of saving that we are starting up the entity
|
|
|
|
|
// and in that case we need to buffer messages until that completes
|
|
|
|
|
refById.get(id) match {
|
|
|
|
|
case Some(actor) =>
|
|
|
|
|
// not using remember entities or write is in progress for other entity and this entity is running already
|
|
|
|
|
// go ahead and deliver
|
|
|
|
|
if (VerboseDebug) log.debug("Delivering message of type [{}] to [{}]", payload.getClass, id)
|
|
|
|
|
touchLastMessageTimestamp(id)
|
|
|
|
|
actor.tell(payload, snd)
|
|
|
|
|
case None =>
|
|
|
|
|
if (entityIds.contains(id)) {
|
|
|
|
|
// No entity actor running but id is in set of entities, this can happen in two scenarios:
|
|
|
|
|
// 1. we are starting up and the entity id is remembered but not yet started
|
|
|
|
|
// 2. the entity is stopped but not passivated, and should be restarted
|
|
|
|
|
|
|
|
|
|
// FIXME won't this potentially lead to not remembered for case 2?
|
|
|
|
|
if (VerboseDebug)
|
|
|
|
|
log.debug(
|
|
|
|
|
"Delivering message of type [{}] to [{}] (starting because known but not running)",
|
|
|
|
|
payload.getClass,
|
|
|
|
|
id)
|
|
|
|
|
val actor = getOrCreateEntity(id)
|
|
|
|
|
touchLastMessageTimestamp(id)
|
|
|
|
|
actor.tell(payload, snd)
|
|
|
|
|
|
|
|
|
|
} else {
|
|
|
|
|
if (entityIdWaitingForWrite.isEmpty) {
|
|
|
|
|
// No actor and id is unknown, start actor and deliver message when started
|
|
|
|
|
// Note; we only do this if remembering, otherwise the buffer is an overhead
|
|
|
|
|
if (VerboseDebug)
|
|
|
|
|
log.debug("Buffering message [{}] to [{}] and starting actor", payload.getClass, id)
|
|
|
|
|
appendToMessageBuffer(id, msg, snd)
|
|
|
|
|
waitForAsyncWrite(id, RememberEntitiesShardStore.AddEntity(id))(sendMsgBuffer)
|
|
|
|
|
} else {
|
|
|
|
|
// we'd need to start the entity but a start/stop write is already in progress
|
|
|
|
|
// see waitForAsyncWrite for unstash
|
|
|
|
|
if (VerboseDebug)
|
|
|
|
|
log.debug(
|
|
|
|
|
"Stashing message [{}] to [{}] because of write in progress for [{}]",
|
|
|
|
|
payload.getClass,
|
|
|
|
|
id,
|
|
|
|
|
entityIdWaitingForWrite.get)
|
|
|
|
|
stash()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2019-05-21 17:29:11 +02:00
|
|
|
@InternalStableApi
|
2019-05-07 08:50:32 +02:00
|
|
|
def getOrCreateEntity(id: EntityId): ActorRef = {
|
2020-04-03 14:57:49 +02:00
|
|
|
refById.get(id) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Some(child) => child
|
|
|
|
|
case None =>
|
2020-04-03 14:57:49 +02:00
|
|
|
val name = URLEncoder.encode(id, "utf-8")
|
2018-12-06 14:49:47 +01:00
|
|
|
val a = context.watch(context.actorOf(entityProps(id), name))
|
2020-04-03 14:57:49 +02:00
|
|
|
log.debug("Started entity [{}] with entity id [{}] in shard [{}]", a, id, shardId)
|
2018-12-06 14:49:47 +01:00
|
|
|
idByRef = idByRef.updated(a, id)
|
|
|
|
|
refById = refById.updated(id, a)
|
2020-04-03 14:57:49 +02:00
|
|
|
entityIds = entityIds + id
|
2019-05-07 08:50:32 +02:00
|
|
|
touchLastMessageTimestamp(id)
|
2018-12-06 14:49:47 +01:00
|
|
|
a
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
2018-11-06 19:15:23 +01:00
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
// ===== buffering while busy saving a start or stop when remembering entities =====
|
|
|
|
|
def appendToMessageBuffer(id: EntityId, msg: Any, snd: ActorRef): Unit = {
|
|
|
|
|
if (messageBuffers.totalSize >= bufferSize) {
|
|
|
|
|
if (log.isDebugEnabled)
|
|
|
|
|
log.debug("Buffer is full, dropping message of type [{}] for entity [{}]", msg.getClass.getName, id)
|
|
|
|
|
context.system.deadLetters ! msg
|
2017-01-18 16:28:24 +01:00
|
|
|
} else {
|
2020-04-03 14:57:49 +02:00
|
|
|
if (log.isDebugEnabled)
|
|
|
|
|
log.debug("Message of type [{}] for entity [{}] buffered", msg.getClass.getName, id)
|
|
|
|
|
messageBuffers.append(id, msg, snd)
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
// After entity started
|
|
|
|
|
def sendMsgBuffer(entityId: EntityId): Unit = {
|
|
|
|
|
//Get the buffered messages and remove the buffer
|
|
|
|
|
val messages = messageBuffers.getOrEmpty(entityId)
|
|
|
|
|
messageBuffers.remove(entityId)
|
2017-01-18 16:28:24 +01:00
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
if (messages.nonEmpty) {
|
|
|
|
|
getOrCreateEntity(entityId)
|
|
|
|
|
log.debug("Sending message buffer for entity [{}] ([{}] messages)", entityId, messages.size)
|
|
|
|
|
//Now there is no deliveryBuffer we can try to redeliver
|
|
|
|
|
// and as the child exists, the message will be directly forwarded
|
|
|
|
|
messages.foreach {
|
|
|
|
|
case (msg, snd) => deliverMessage(msg, snd, OptionVal.None)
|
|
|
|
|
}
|
|
|
|
|
touchLastMessageTimestamp(entityId)
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
def dropBufferFor(entityId: EntityId): Unit = {
|
|
|
|
|
if (log.isDebugEnabled) {
|
|
|
|
|
val messages = messageBuffers.getOrEmpty(entityId)
|
|
|
|
|
if (messages.nonEmpty)
|
|
|
|
|
log.debug("Dropping [{}] buffered messages", entityId, messages.size)
|
2019-03-11 10:38:24 +01:00
|
|
|
}
|
2020-04-03 14:57:49 +02:00
|
|
|
messageBuffers.remove(entityId)
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
private def rememberEntityStoreCrashed(msg: RememberEntityStoreCrashed): Unit = {
|
|
|
|
|
throw new RuntimeException(s"Remember entities store [${msg.store}] crashed")
|
2019-05-07 08:50:32 +02:00
|
|
|
}
|
|
|
|
|
|
2020-04-03 14:57:49 +02:00
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
passivateIdleTask.foreach(_.cancel())
|
2016-08-01 09:46:09 +01:00
|
|
|
}
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|