2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2019-01-02 18:55:26 +08:00
|
|
|
* Copyright (C) 2009-2019 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-03-28 13:24:46 +01:00
|
|
|
import akka.actor.{
|
|
|
|
|
Actor,
|
|
|
|
|
ActorLogging,
|
|
|
|
|
ActorRef,
|
|
|
|
|
ActorSystem,
|
|
|
|
|
DeadLetterSuppression,
|
|
|
|
|
Deploy,
|
|
|
|
|
NoSerializationVerificationNeeded,
|
|
|
|
|
Props,
|
|
|
|
|
Stash,
|
|
|
|
|
Terminated,
|
|
|
|
|
Timers
|
|
|
|
|
}
|
2018-12-06 14:49:47 +01:00
|
|
|
import akka.util.{ ConstantFun, MessageBufferMap }
|
2016-08-01 09:46:09 +01:00
|
|
|
import scala.concurrent.Future
|
2019-03-28 13:24:46 +01:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
import akka.cluster.Cluster
|
|
|
|
|
import akka.cluster.ddata.ORSet
|
|
|
|
|
import akka.cluster.ddata.ORSetKey
|
|
|
|
|
import akka.cluster.ddata.Replicator._
|
2018-09-25 13:38:18 +01:00
|
|
|
import akka.persistence._
|
2019-03-28 13:24:46 +01:00
|
|
|
import akka.util.PrettyDuration._
|
|
|
|
|
import akka.coordination.lease.scaladsl.{ Lease, LeaseProvider }
|
|
|
|
|
import akka.pattern.pipe
|
2018-11-06 19:15:23 +01:00
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
import akka.cluster.sharding.ShardRegion.ShardInitialized
|
|
|
|
|
|
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
|
|
|
|
|
*/
|
|
|
|
|
sealed trait ShardCommand
|
|
|
|
|
|
|
|
|
|
/**
|
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.
|
|
|
|
|
*/
|
|
|
|
|
final case class RestartEntity(entity: EntityId) extends ShardCommand
|
|
|
|
|
|
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.
|
|
|
|
|
*/
|
|
|
|
|
final case class RestartEntities(entity: Set[EntityId]) extends ShardCommand
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
/**
|
|
|
|
|
* A case class which represents a state change for the Shard
|
|
|
|
|
*/
|
2015-08-17 16:11:52 +02:00
|
|
|
sealed trait StateChange extends ClusterShardingSerializable {
|
|
|
|
|
val entityId: EntityId
|
|
|
|
|
}
|
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
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
/**
|
|
|
|
|
* `State` change for starting an entity in this `Shard`
|
|
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L) final case class EntityStarted(entityId: EntityId) extends StateChange
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* `State` change for an entity which has terminated.
|
|
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L) final case class EntityStopped(entityId: EntityId) extends StateChange
|
|
|
|
|
|
2015-09-23 11:24:39 +02:00
|
|
|
@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"
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
object State {
|
|
|
|
|
val Empty = State()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Persistent state of the Shard.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@SerialVersionUID(1L) final case class State private[akka] (entities: Set[EntityId] = Set.empty)
|
|
|
|
|
extends ClusterShardingSerializable
|
2015-06-22 08:09:10 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Factory method for the [[akka.actor.Props]] of the [[Shard]] actor.
|
2015-06-22 08:54:42 +02:00
|
|
|
* If `settings.rememberEntities` is enabled the `PersistentShard`
|
|
|
|
|
* subclass is used, otherwise `Shard`.
|
2015-06-22 08:09:10 +02:00
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
def props(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
entityProps: String => Props,
|
|
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
|
|
|
|
handOffStopMessage: Any,
|
|
|
|
|
replicator: ActorRef,
|
|
|
|
|
majorityMinCap: Int): Props = {
|
2017-01-18 16:28:24 +01:00
|
|
|
if (settings.rememberEntities && settings.stateStoreMode == ClusterShardingSettings.StateStoreModeDData) {
|
2019-03-11 10:38:24 +01:00
|
|
|
Props(
|
2019-03-13 10:56:20 +01:00
|
|
|
new DDataShard(
|
|
|
|
|
typeName,
|
|
|
|
|
shardId,
|
|
|
|
|
entityProps,
|
|
|
|
|
settings,
|
|
|
|
|
extractEntityId,
|
|
|
|
|
extractShardId,
|
|
|
|
|
handOffStopMessage,
|
|
|
|
|
replicator,
|
|
|
|
|
majorityMinCap)).withDeploy(Deploy.local)
|
2017-01-18 16:28:24 +01:00
|
|
|
} else if (settings.rememberEntities && settings.stateStoreMode == ClusterShardingSettings.StateStoreModePersistence)
|
2019-03-11 10:38:24 +01:00
|
|
|
Props(
|
2019-03-13 10:56:20 +01:00
|
|
|
new PersistentShard(
|
|
|
|
|
typeName,
|
|
|
|
|
shardId,
|
|
|
|
|
entityProps,
|
|
|
|
|
settings,
|
|
|
|
|
extractEntityId,
|
|
|
|
|
extractShardId,
|
|
|
|
|
handOffStopMessage)).withDeploy(Deploy.local)
|
2015-06-22 08:54:42 +02:00
|
|
|
else
|
2017-12-07 17:49:29 +01:00
|
|
|
Props(new Shard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage))
|
2015-06-22 08:54:42 +02:00
|
|
|
.withDeploy(Deploy.local)
|
|
|
|
|
}
|
2018-11-06 19:15:23 +01:00
|
|
|
|
|
|
|
|
private case object PassivateIdleTick extends NoSerializationVerificationNeeded
|
|
|
|
|
|
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]]
|
|
|
|
|
*/
|
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,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
|
|
|
|
handOffStopMessage: Any)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends Actor
|
2019-03-28 13:24:46 +01:00
|
|
|
with ActorLogging
|
|
|
|
|
with Timers {
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
import ShardRegion.{ handOffStopperProps, EntityId, Msg, Passivate, ShardInitialized }
|
2015-06-22 08:09:10 +02:00
|
|
|
import ShardCoordinator.Internal.{ HandOff, ShardStopped }
|
2017-01-18 16:28:24 +01:00
|
|
|
import Shard._
|
2015-06-22 08:09:10 +02:00
|
|
|
import akka.cluster.sharding.ShardCoordinator.Internal.CoordinatorMessage
|
|
|
|
|
import akka.cluster.sharding.ShardRegion.ShardRegionCommand
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
|
|
|
|
|
var state = State.Empty
|
|
|
|
|
var idByRef = Map.empty[ActorRef, EntityId]
|
|
|
|
|
var refById = Map.empty[EntityId, ActorRef]
|
2018-11-06 19:15:23 +01:00
|
|
|
var lastMessageTimestamp = Map.empty[EntityId, Long]
|
2015-06-22 08:09:10 +02:00
|
|
|
var passivating = Set.empty[ActorRef]
|
2017-03-15 13:31:47 +01:00
|
|
|
val messageBuffers = new MessageBufferMap[EntityId]
|
2015-06-22 08:09:10 +02:00
|
|
|
|
|
|
|
|
var handOffStopper: Option[ActorRef] = None
|
|
|
|
|
|
2018-11-06 19:15:23 +01:00
|
|
|
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
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
val lease = settings.leaseSettings.map(
|
|
|
|
|
ls =>
|
|
|
|
|
LeaseProvider(context.system).getLease(
|
|
|
|
|
s"${context.system.name}-shard-$typeName-$shardId",
|
|
|
|
|
ls.leaseImplementation,
|
|
|
|
|
Cluster(context.system).selfAddress.hostPort))
|
|
|
|
|
|
|
|
|
|
val leaseRetryInterval = settings.leaseSettings match {
|
|
|
|
|
case Some(l) => l.leaseRetryInterval
|
|
|
|
|
case None => 5.seconds // not used
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
acquireLeaseIfNeeded()
|
|
|
|
|
}
|
2015-08-20 13:24:39 +03:00
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
/**
|
|
|
|
|
* Will call onLeaseAcquired when completed, also when lease isn't used
|
|
|
|
|
*/
|
|
|
|
|
def acquireLeaseIfNeeded(): Unit = {
|
|
|
|
|
lease match {
|
|
|
|
|
case Some(l) =>
|
|
|
|
|
tryGetLease(l)
|
|
|
|
|
context.become(awaitingLease())
|
|
|
|
|
case None =>
|
|
|
|
|
onLeaseAcquired()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Override to execute logic once the lease has been acquired
|
|
|
|
|
// Will be called on the actor thread
|
|
|
|
|
def onLeaseAcquired(): Unit = {
|
|
|
|
|
log.debug("Shard initialized")
|
|
|
|
|
context.parent ! ShardInitialized(shardId)
|
|
|
|
|
context.become(receiveCommand)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def tryGetLease(l: Lease) = {
|
|
|
|
|
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)
|
|
|
|
|
}
|
2015-08-20 13:24:39 +03:00
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
def processChange[E <: StateChange](event: E)(handler: E => Unit): Unit =
|
2015-06-22 08:54:42 +02:00
|
|
|
handler(event)
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
def receive = receiveCommand
|
2015-06-22 08:09:10 +02:00
|
|
|
|
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
|
|
|
|
|
def awaitingLease(): Receive = {
|
|
|
|
|
case LeaseAcquireResult(true, _) =>
|
|
|
|
|
log.debug("Acquired lease")
|
|
|
|
|
onLeaseAcquired()
|
|
|
|
|
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)
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
def receiveCommand: Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Terminated(ref) => receiveTerminated(ref)
|
|
|
|
|
case msg: CoordinatorMessage => receiveCoordinatorMessage(msg)
|
|
|
|
|
case msg: ShardCommand => receiveShardCommand(msg)
|
|
|
|
|
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)
|
2019-02-09 15:25:39 +01:00
|
|
|
case msg if extractEntityId.isDefinedAt(msg) => deliverMessage(msg, sender())
|
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
|
|
|
|
|
log.error("Shard type [{}] id [{}] lease lost. Reason: {}", typeName, shardId, msg.reason)
|
|
|
|
|
// Stop entities ASAP rather than send termination message
|
|
|
|
|
context.stop(self)
|
|
|
|
|
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
def receiveShardCommand(msg: ShardCommand): Unit = msg match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case RestartEntity(id) => getOrCreateEntity(id)
|
|
|
|
|
case RestartEntities(ids) => restartEntities(ids)
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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)
|
2018-11-06 19:15:23 +01:00
|
|
|
if (passivateIdleTask.isDefined) {
|
|
|
|
|
lastMessageTimestamp = lastMessageTimestamp.updated(start.entityId, System.nanoTime())
|
|
|
|
|
}
|
2019-03-13 10:56:20 +01:00
|
|
|
getOrCreateEntity(
|
|
|
|
|
start.entityId,
|
|
|
|
|
_ =>
|
|
|
|
|
processChange(EntityStarted(start.entityId))(_ =>
|
|
|
|
|
requester ! ShardRegion.StartEntityAck(start.entityId, shardId)))
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receiveStartEntityAck(ack: ShardRegion.StartEntityAck): Unit = {
|
|
|
|
|
if (ack.shardId != shardId && state.entities.contains(ack.entityId)) {
|
|
|
|
|
log.debug("Entity [{}] previously owned by shard [{}] started in shard [{}]", ack.entityId, shardId, ack.shardId)
|
2019-02-09 15:25:39 +01:00
|
|
|
processChange(EntityStopped(ack.entityId)) { _ =>
|
2017-05-22 10:08:18 +02:00
|
|
|
state = state.copy(state.entities - ack.entityId)
|
|
|
|
|
messageBuffers.remove(ack.entityId)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def restartEntities(ids: Set[EntityId]): Unit = {
|
2017-05-22 13:14:43 +02:00
|
|
|
context.actorOf(RememberEntityStarter.props(context.parent, typeName, shardId, ids, settings, sender()))
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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)
|
|
|
|
|
case GetShardStats => sender() ! ShardStats(shardId, state.entities.size)
|
2015-09-23 11:24:39 +02:00
|
|
|
}
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
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)
|
|
|
|
|
|
|
|
|
|
if (state.entities.nonEmpty) {
|
2018-12-04 20:50:58 +03:30
|
|
|
val entityHandOffTimeout = (settings.tuningParameters.handOffTimeout - 5.seconds).max(1.seconds)
|
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
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
def entityTerminated(ref: ActorRef): Unit = {
|
|
|
|
|
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) {
|
2015-06-22 08:54:42 +02:00
|
|
|
log.debug("Starting entity [{}] again, there are buffered messages for it", id)
|
|
|
|
|
sendMsgBuffer(EntityStarted(id))
|
|
|
|
|
} else {
|
|
|
|
|
processChange(EntityStopped(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
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def passivate(entity: ActorRef, stopMessage: Any): Unit = {
|
|
|
|
|
idByRef.get(entity) match {
|
2019-03-11 10:38:24 +01:00
|
|
|
case Some(id) =>
|
|
|
|
|
if (!messageBuffers.contains(id)) {
|
|
|
|
|
passivating = passivating + entity
|
|
|
|
|
messageBuffers.add(id)
|
|
|
|
|
entity ! stopMessage
|
|
|
|
|
} else {
|
|
|
|
|
log.debug("Passivation already in progress for {}. Not sending stopMessage back to entity.", entity)
|
|
|
|
|
}
|
2019-02-09 15:25:39 +01:00
|
|
|
case None => log.debug("Unknown entity {}. Not sending stopMessage back to entity.", entity)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2018-11-06 19:15:23 +01:00
|
|
|
def passivateIdleEntities(): Unit = {
|
|
|
|
|
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))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
// EntityStopped handler
|
2015-06-22 08:09:10 +02:00
|
|
|
def passivateCompleted(event: EntityStopped): Unit = {
|
2017-05-15 15:27:53 +02:00
|
|
|
log.debug("Entity stopped after passivation [{}]", event.entityId)
|
2015-06-22 08:09:10 +02:00
|
|
|
state = state.copy(state.entities - event.entityId)
|
2017-03-15 13:31:47 +01:00
|
|
|
messageBuffers.remove(event.entityId)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
// EntityStarted handler
|
2015-06-22 08:09:10 +02:00
|
|
|
def sendMsgBuffer(event: EntityStarted): Unit = {
|
|
|
|
|
//Get the buffered messages and remove the buffer
|
2017-03-15 13:31:47 +01:00
|
|
|
val messages = messageBuffers.getOrEmpty(event.entityId)
|
|
|
|
|
messageBuffers.remove(event.entityId)
|
2015-06-22 08:09:10 +02:00
|
|
|
|
|
|
|
|
if (messages.nonEmpty) {
|
|
|
|
|
log.debug("Sending message buffer for entity [{}] ([{}] messages)", event.entityId, messages.size)
|
2018-12-06 14:49:47 +01:00
|
|
|
getOrCreateEntity(event.entityId)
|
2015-06-22 08:09:10 +02:00
|
|
|
//Now there is no deliveryBuffer we can try to redeliver
|
|
|
|
|
// and as the child exists, the message will be directly forwarded
|
2017-03-15 13:31:47 +01:00
|
|
|
messages.foreach {
|
2019-02-09 15:25:39 +01:00
|
|
|
case (msg, snd) => deliverMessage(msg, snd)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def deliverMessage(msg: Any, snd: ActorRef): Unit = {
|
|
|
|
|
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 =>
|
2018-11-06 19:15:23 +01:00
|
|
|
// in case it was wrapped, used in Typed
|
|
|
|
|
receiveStartEntity(start)
|
2019-02-09 15:25:39 +01:00
|
|
|
case _ =>
|
2018-11-06 19:15:23 +01:00
|
|
|
messageBuffers.contains(id) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case false => deliverTo(id, msg, payload, snd)
|
2018-11-06 19:15:23 +01:00
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case true if messageBuffers.totalSize >= bufferSize =>
|
2018-11-06 19:15:23 +01:00
|
|
|
log.debug("Buffer is full, dropping message for entity [{}]", id)
|
|
|
|
|
context.system.deadLetters ! msg
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case true =>
|
2018-11-06 19:15:23 +01:00
|
|
|
log.debug("Message for entity [{}] buffered", id)
|
|
|
|
|
messageBuffers.append(id, msg, snd)
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def deliverTo(id: EntityId, msg: Any, payload: Msg, snd: ActorRef): Unit = {
|
2018-11-06 19:15:23 +01:00
|
|
|
if (passivateIdleTask.isDefined) {
|
|
|
|
|
lastMessageTimestamp = lastMessageTimestamp.updated(id, System.nanoTime())
|
|
|
|
|
}
|
2018-12-06 14:49:47 +01:00
|
|
|
getOrCreateEntity(id).tell(payload, snd)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
def getOrCreateEntity(id: EntityId, onCreate: ActorRef => Unit = ConstantFun.scalaAnyToUnit): ActorRef = {
|
2015-06-22 08:09:10 +02:00
|
|
|
val name = URLEncoder.encode(id, "utf-8")
|
2018-12-06 14:49:47 +01:00
|
|
|
context.child(name) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Some(child) => child
|
|
|
|
|
case None =>
|
2018-12-06 14:49:47 +01:00
|
|
|
log.debug("Starting entity [{}] in shard [{}]", id, shardId)
|
|
|
|
|
val a = context.watch(context.actorOf(entityProps(id), name))
|
|
|
|
|
idByRef = idByRef.updated(a, id)
|
|
|
|
|
refById = refById.updated(id, a)
|
|
|
|
|
if (passivateIdleTask.isDefined) {
|
|
|
|
|
lastMessageTimestamp += (id -> System.nanoTime())
|
|
|
|
|
}
|
|
|
|
|
state = state.copy(state.entities + id)
|
|
|
|
|
onCreate(a)
|
|
|
|
|
a
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
2018-11-06 19:15:23 +01:00
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
passivateIdleTask.foreach(_.cancel())
|
|
|
|
|
}
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
2015-06-22 08:54:42 +02:00
|
|
|
|
2017-05-22 10:08:18 +02:00
|
|
|
private[akka] object RememberEntityStarter {
|
2019-03-13 10:56:20 +01:00
|
|
|
def props(
|
|
|
|
|
region: ActorRef,
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
ids: Set[ShardRegion.EntityId],
|
|
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
requestor: ActorRef) =
|
2017-05-22 13:14:43 +02:00
|
|
|
Props(new RememberEntityStarter(region, typeName, shardId, ids, settings, requestor))
|
|
|
|
|
|
|
|
|
|
private case object Tick extends NoSerializationVerificationNeeded
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API: Actor responsible for starting entities when rememberEntities is enabled
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
private[akka] class RememberEntityStarter(
|
|
|
|
|
region: ActorRef,
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
ids: Set[ShardRegion.EntityId],
|
|
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
requestor: ActorRef)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends Actor
|
|
|
|
|
with ActorLogging {
|
2017-05-22 10:08:18 +02:00
|
|
|
|
|
|
|
|
import context.dispatcher
|
2017-05-22 13:14:43 +02:00
|
|
|
import RememberEntityStarter.Tick
|
2017-05-22 10:08:18 +02:00
|
|
|
|
|
|
|
|
var waitingForAck = ids
|
|
|
|
|
|
|
|
|
|
sendStart(ids)
|
|
|
|
|
|
|
|
|
|
val tickTask = {
|
|
|
|
|
val resendInterval = settings.tuningParameters.retryInterval
|
|
|
|
|
context.system.scheduler.schedule(resendInterval, resendInterval, self, Tick)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def sendStart(ids: Set[ShardRegion.EntityId]): Unit = {
|
2018-12-06 14:49:47 +01:00
|
|
|
// these go through the region rather the directly to the shard
|
|
|
|
|
// so that shard mapping changes are picked up
|
2019-02-09 15:25:39 +01:00
|
|
|
ids.foreach(id => region ! ShardRegion.StartEntity(id))
|
2017-05-22 10:08:18 +02:00
|
|
|
}
|
|
|
|
|
|
2018-12-06 14:49:47 +01:00
|
|
|
override def receive: Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case ack: ShardRegion.StartEntityAck =>
|
2017-05-22 10:08:18 +02:00
|
|
|
waitingForAck -= ack.entityId
|
|
|
|
|
// inform whoever requested the start that it happened
|
|
|
|
|
requestor ! ack
|
|
|
|
|
if (waitingForAck.isEmpty) context.stop(self)
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case Tick =>
|
2017-05-22 10:08:18 +02:00
|
|
|
sendStart(waitingForAck)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
tickTask.cancel()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API: Common things for PersistentShard and DDataShard
|
|
|
|
|
*/
|
2018-12-06 14:49:47 +01:00
|
|
|
private[akka] trait RememberingShard {
|
2019-02-09 15:25:39 +01:00
|
|
|
selfType: Shard =>
|
2018-12-06 14:49:47 +01:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
import ShardRegion.{ EntityId, Msg }
|
|
|
|
|
import Shard._
|
|
|
|
|
import akka.pattern.pipe
|
|
|
|
|
|
|
|
|
|
protected val settings: ClusterShardingSettings
|
|
|
|
|
|
|
|
|
|
protected val rememberedEntitiesRecoveryStrategy: EntityRecoveryStrategy = {
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
entityRecoveryStrategy match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case "all" => EntityRecoveryStrategy.allStrategy()
|
2019-03-11 10:38:24 +01:00
|
|
|
case "constant" =>
|
2019-03-13 10:56:20 +01:00
|
|
|
EntityRecoveryStrategy.constantStrategy(
|
|
|
|
|
context.system,
|
|
|
|
|
entityRecoveryConstantRateStrategyFrequency,
|
|
|
|
|
entityRecoveryConstantRateStrategyNumberOfEntities)
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
protected def restartRememberedEntities(): Unit = {
|
2019-02-09 15:25:39 +01:00
|
|
|
rememberedEntitiesRecoveryStrategy.recoverEntities(state.entities).foreach { scheduledRecovery =>
|
2017-01-18 16:28:24 +01:00
|
|
|
import context.dispatcher
|
|
|
|
|
scheduledRecovery.filter(_.nonEmpty).map(RestartEntities).pipeTo(self)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def entityTerminated(ref: ActorRef): Unit = {
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
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) {
|
2017-01-18 16:28:24 +01:00
|
|
|
//Note; because we're not persisting the EntityStopped, we don't need
|
|
|
|
|
// to persist the EntityStarted either.
|
|
|
|
|
log.debug("Starting entity [{}] again, there are buffered messages for it", id)
|
|
|
|
|
sendMsgBuffer(EntityStarted(id))
|
|
|
|
|
} else {
|
|
|
|
|
if (!passivating.contains(ref)) {
|
|
|
|
|
log.debug("Entity [{}] stopped without passivating, will restart after backoff", id)
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
context.system.scheduler.scheduleOnce(entityRestartBackoff, self, RestartEntity(id))
|
|
|
|
|
} else processChange(EntityStopped(id))(passivateCompleted)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
passivating = passivating - ref
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def deliverTo(id: EntityId, msg: Any, payload: Msg, snd: ActorRef): Unit = {
|
|
|
|
|
val name = URLEncoder.encode(id, "utf-8")
|
|
|
|
|
context.child(name) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Some(actor) =>
|
2017-01-18 16:28:24 +01:00
|
|
|
actor.tell(payload, snd)
|
2019-02-09 15:25:39 +01:00
|
|
|
case None =>
|
2018-07-10 09:24:52 -06:00
|
|
|
if (state.entities.contains(id)) {
|
|
|
|
|
require(!messageBuffers.contains(id), s"Message buffers contains id [$id].")
|
2018-12-06 14:49:47 +01:00
|
|
|
getOrCreateEntity(id).tell(payload, snd)
|
2018-07-10 09:24:52 -06:00
|
|
|
} else {
|
|
|
|
|
//Note; we only do this if remembering, otherwise the buffer is an overhead
|
|
|
|
|
messageBuffers.append(id, msg, snd)
|
|
|
|
|
processChange(EntityStarted(id))(sendMsgBuffer)
|
|
|
|
|
}
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* This actor creates children entity actors on demand that it is told to be
|
2017-01-18 16:28:24 +01:00
|
|
|
* responsible for. It is used when `rememberEntities` is enabled and
|
|
|
|
|
* `state-store-mode=persistence`.
|
2015-06-22 08:54:42 +02:00
|
|
|
*
|
|
|
|
|
* @see [[ClusterSharding$ ClusterSharding extension]]
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
private[akka] class PersistentShard(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
entityProps: String => Props,
|
|
|
|
|
override val settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
|
|
|
|
handOffStopMessage: Any)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends Shard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage)
|
|
|
|
|
with RememberingShard
|
|
|
|
|
with PersistentActor
|
|
|
|
|
with ActorLogging {
|
2015-06-22 08:54:42 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
import Shard._
|
2015-06-22 08:54:42 +02:00
|
|
|
import settings.tuningParameters._
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
// override to not acquire the lease on start up, acquire after persistent recovery
|
|
|
|
|
}
|
|
|
|
|
|
2017-02-21 16:17:19 +03:00
|
|
|
override def persistenceId = s"/sharding/${typeName}Shard/$shardId"
|
2015-06-22 08:54:42 +02:00
|
|
|
|
|
|
|
|
override def journalPluginId: String = settings.journalPluginId
|
|
|
|
|
|
|
|
|
|
override def snapshotPluginId: String = settings.snapshotPluginId
|
|
|
|
|
|
|
|
|
|
override def receive = receiveCommand
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
override def processChange[E <: StateChange](event: E)(handler: E => Unit): Unit = {
|
2015-06-22 08:54:42 +02:00
|
|
|
saveSnapshotWhenNeeded()
|
|
|
|
|
persist(event)(handler)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def saveSnapshotWhenNeeded(): Unit = {
|
2016-05-05 16:21:24 +02:00
|
|
|
if (lastSequenceNr % snapshotAfter == 0 && lastSequenceNr != 0) {
|
2015-06-22 08:54:42 +02:00
|
|
|
log.debug("Saving snapshot, sequence number [{}]", snapshotSequenceNr)
|
|
|
|
|
saveSnapshot(state)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def receiveRecover: Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case EntityStarted(id) => state = state.copy(state.entities + id)
|
|
|
|
|
case EntityStopped(id) => state = state.copy(state.entities - id)
|
|
|
|
|
case SnapshotOffer(_, snapshot: State) => state = snapshot
|
|
|
|
|
case RecoveryCompleted =>
|
2019-03-28 13:24:46 +01:00
|
|
|
acquireLeaseIfNeeded() // onLeaseAcquired called when completed
|
2017-01-18 16:28:24 +01:00
|
|
|
log.debug("PersistentShard recovery completed shard [{}] with [{}] entities", shardId, state.entities.size)
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
override def onLeaseAcquired(): Unit = {
|
|
|
|
|
log.debug("Shard initialized")
|
|
|
|
|
context.parent ! ShardInitialized(shardId)
|
|
|
|
|
context.become(receiveCommand)
|
|
|
|
|
restartRememberedEntities()
|
|
|
|
|
unstashAll()
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
override def receiveCommand: Receive =
|
|
|
|
|
({
|
|
|
|
|
case e: SaveSnapshotSuccess =>
|
|
|
|
|
log.debug("PersistentShard snapshot saved successfully")
|
|
|
|
|
internalDeleteMessagesBeforeSnapshot(e, keepNrOfBatches, snapshotAfter)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
case SaveSnapshotFailure(_, reason) =>
|
|
|
|
|
log.warning("PersistentShard snapshot failure: [{}]", reason.getMessage)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
case DeleteMessagesSuccess(toSequenceNr) =>
|
|
|
|
|
val deleteTo = toSequenceNr - 1
|
|
|
|
|
val deleteFrom = math.max(0, deleteTo - (keepNrOfBatches * snapshotAfter))
|
2019-03-13 10:56:20 +01:00
|
|
|
log.debug(
|
|
|
|
|
"PersistentShard messages to [{}] deleted successfully. Deleting snapshots from [{}] to [{}]",
|
|
|
|
|
toSequenceNr,
|
|
|
|
|
deleteFrom,
|
|
|
|
|
deleteTo)
|
2019-03-11 10:38:24 +01:00
|
|
|
deleteSnapshots(SnapshotSelectionCriteria(minSequenceNr = deleteFrom, maxSequenceNr = deleteTo))
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
case DeleteMessagesFailure(reason, toSequenceNr) =>
|
|
|
|
|
log.warning("PersistentShard messages to [{}] deletion failure: [{}]", toSequenceNr, reason.getMessage)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
case DeleteSnapshotsSuccess(m) =>
|
|
|
|
|
log.debug("PersistentShard snapshots matching [{}] deleted successfully", m)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
case DeleteSnapshotsFailure(m, reason) =>
|
|
|
|
|
log.warning("PersistentShard snapshots matching [{}] deletion failure: [{}]", m, reason.getMessage)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
}: Receive).orElse(super.receiveCommand)
|
2016-03-15 15:35:25 +01:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
2015-06-22 08:54:42 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* This actor creates children entity actors on demand that it is told to be
|
|
|
|
|
* responsible for. It is used when `rememberEntities` is enabled and
|
|
|
|
|
* `state-store-mode=ddata`.
|
|
|
|
|
*
|
|
|
|
|
* @see [[ClusterSharding$ ClusterSharding extension]]
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
private[akka] class DDataShard(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
entityProps: String => Props,
|
|
|
|
|
override val settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
|
|
|
|
handOffStopMessage: Any,
|
|
|
|
|
replicator: ActorRef,
|
|
|
|
|
majorityMinCap: Int)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends Shard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage)
|
|
|
|
|
with RememberingShard
|
|
|
|
|
with Stash
|
|
|
|
|
with ActorLogging {
|
2017-01-18 16:28:24 +01:00
|
|
|
|
2018-03-16 19:08:29 +08:00
|
|
|
import ShardRegion.EntityId
|
2017-01-18 16:28:24 +01:00
|
|
|
import Shard._
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
private val readMajority = ReadMajority(settings.tuningParameters.waitingForStateTimeout, majorityMinCap)
|
2017-01-24 14:40:12 +01:00
|
|
|
private val writeMajority = WriteMajority(settings.tuningParameters.updatingStateTimeout, majorityMinCap)
|
2017-01-18 16:28:24 +01:00
|
|
|
private val maxUpdateAttempts = 3
|
|
|
|
|
|
|
|
|
|
implicit private val node = Cluster(context.system)
|
|
|
|
|
|
|
|
|
|
// The default maximum-frame-size is 256 KiB with Artery.
|
2017-02-20 14:40:54 +01:00
|
|
|
// When using entity identifiers with 36 character strings (e.g. UUID.randomUUID).
|
|
|
|
|
// By splitting the elements over 5 keys we can support 10000 entities per shard.
|
|
|
|
|
// The Gossip message size of 5 ORSet with 2000 ids is around 200 KiB.
|
2017-01-18 16:28:24 +01:00
|
|
|
// This is by intention not configurable because it's important to have the same
|
|
|
|
|
// configuration on each node.
|
|
|
|
|
private val numberOfKeys = 5
|
|
|
|
|
private val stateKeys: Array[ORSetKey[EntityId]] =
|
2019-02-09 15:25:39 +01:00
|
|
|
Array.tabulate(numberOfKeys)(i => ORSetKey[EntityId](s"shard-${typeName}-${shardId}-$i"))
|
2017-01-18 16:28:24 +01:00
|
|
|
|
|
|
|
|
private def key(entityId: EntityId): ORSetKey[EntityId] = {
|
2018-05-01 16:43:56 +02:00
|
|
|
val i = (math.abs(entityId.hashCode % numberOfKeys))
|
2017-01-18 16:28:24 +01:00
|
|
|
stateKeys(i)
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
override def onLeaseAcquired(): Unit = {
|
|
|
|
|
log.info("Lease Acquired. Getting state from DData")
|
|
|
|
|
getState()
|
|
|
|
|
context.become(waitingForState(Set.empty))
|
|
|
|
|
}
|
2015-06-22 08:54:42 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
private def getState(): Unit = {
|
2019-02-09 15:25:39 +01:00
|
|
|
(0 until numberOfKeys).map { i =>
|
2017-01-24 14:40:12 +01:00
|
|
|
replicator ! Get(stateKeys(i), readMajority, Some(i))
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
override def receive = waitingForState(Set.empty)
|
|
|
|
|
|
|
|
|
|
// This state will stash all commands
|
|
|
|
|
private def waitingForState(gotKeys: Set[Int]): Receive = {
|
|
|
|
|
def receiveOne(i: Int): Unit = {
|
|
|
|
|
val newGotKeys = gotKeys + i
|
2019-03-28 13:24:46 +01:00
|
|
|
if (newGotKeys.size == numberOfKeys) {
|
2017-01-18 16:28:24 +01:00
|
|
|
recoveryCompleted()
|
2019-03-28 13:24:46 +01:00
|
|
|
} else
|
2017-01-18 16:28:24 +01:00
|
|
|
context.become(waitingForState(newGotKeys))
|
2016-08-01 09:46:09 +01:00
|
|
|
}
|
2017-01-18 16:28:24 +01:00
|
|
|
|
|
|
|
|
{
|
2019-02-09 15:25:39 +01:00
|
|
|
case g @ GetSuccess(_, Some(i: Int)) =>
|
2017-01-18 16:28:24 +01:00
|
|
|
val key = stateKeys(i)
|
2019-03-11 10:38:24 +01:00
|
|
|
state = state.copy(entities = state.entities.union(g.get(key).elements))
|
2017-01-18 16:28:24 +01:00
|
|
|
receiveOne(i)
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case GetFailure(_, _) =>
|
2019-03-13 10:56:20 +01:00
|
|
|
log.error(
|
|
|
|
|
"The DDataShard was unable to get an initial state within 'waiting-for-state-timeout': {} millis",
|
|
|
|
|
waitingForStateTimeout.toMillis)
|
2017-01-18 16:28:24 +01:00
|
|
|
// parent ShardRegion supervisor will notice that it terminated and will start it again, after backoff
|
|
|
|
|
context.stop(self)
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case NotFound(_, Some(i: Int)) =>
|
2017-01-18 16:28:24 +01:00
|
|
|
receiveOne(i)
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case _ =>
|
2017-01-18 16:28:24 +01:00
|
|
|
stash()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def recoveryCompleted(): Unit = {
|
|
|
|
|
log.debug("DDataShard recovery completed shard [{}] with [{}] entities", shardId, state.entities.size)
|
2019-03-28 13:24:46 +01:00
|
|
|
context.parent ! ShardInitialized(shardId)
|
2017-01-18 16:28:24 +01:00
|
|
|
context.become(receiveCommand)
|
2019-03-28 13:24:46 +01:00
|
|
|
restartRememberedEntities()
|
|
|
|
|
unstashAll()
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
override def processChange[E <: StateChange](event: E)(handler: E => Unit): Unit = {
|
2017-01-18 16:28:24 +01:00
|
|
|
context.become(waitingForUpdate(event, handler), discardOld = false)
|
|
|
|
|
sendUpdate(event, retryCount = 1)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def sendUpdate(evt: StateChange, retryCount: Int) = {
|
2019-03-11 10:38:24 +01:00
|
|
|
replicator ! Update(key(evt.entityId), ORSet.empty[EntityId], writeMajority, Some((evt, retryCount))) { existing =>
|
|
|
|
|
evt match {
|
|
|
|
|
case EntityStarted(id) => existing + id
|
|
|
|
|
case EntityStopped(id) => existing - id
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
2019-03-11 10:38:24 +01:00
|
|
|
}
|
2017-01-18 16:28:24 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// this state will stash all messages until it receives UpdateSuccess
|
2019-02-09 15:25:39 +01:00
|
|
|
private def waitingForUpdate[E <: StateChange](evt: E, afterUpdateCallback: E => Unit): Receive = {
|
|
|
|
|
case UpdateSuccess(_, Some((`evt`, _))) =>
|
2017-01-18 16:28:24 +01:00
|
|
|
log.debug("The DDataShard state was successfully updated with {}", evt)
|
|
|
|
|
context.unbecome()
|
|
|
|
|
afterUpdateCallback(evt)
|
|
|
|
|
unstashAll()
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case UpdateTimeout(_, Some((`evt`, retryCount: Int))) =>
|
2017-01-18 16:28:24 +01:00
|
|
|
if (retryCount == maxUpdateAttempts) {
|
|
|
|
|
// parent ShardRegion supervisor will notice that it terminated and will start it again, after backoff
|
|
|
|
|
log.error(
|
|
|
|
|
"The DDataShard was unable to update state after {} attempts, within 'updating-state-timeout'={} millis, event={}. " +
|
2019-03-11 10:38:24 +01:00
|
|
|
"Shard will be restarted after backoff.",
|
|
|
|
|
maxUpdateAttempts,
|
|
|
|
|
updatingStateTimeout.toMillis,
|
|
|
|
|
evt)
|
2017-01-18 16:28:24 +01:00
|
|
|
context.stop(self)
|
|
|
|
|
} else {
|
|
|
|
|
log.warning(
|
|
|
|
|
"The DDataShard was unable to update state, attempt {} of {}, within 'updating-state-timeout'={} millis, event={}",
|
2019-03-11 10:38:24 +01:00
|
|
|
retryCount,
|
|
|
|
|
maxUpdateAttempts,
|
|
|
|
|
updatingStateTimeout.toMillis,
|
|
|
|
|
evt)
|
2017-01-18 16:28:24 +01:00
|
|
|
sendUpdate(evt, retryCount + 1)
|
|
|
|
|
}
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
case ModifyFailure(_, error, cause, Some((`evt`, _))) =>
|
2019-03-13 10:56:20 +01:00
|
|
|
log.error(
|
|
|
|
|
cause,
|
|
|
|
|
"The DDataShard was unable to update state with error {} and event {}. Shard will be restarted",
|
|
|
|
|
error,
|
|
|
|
|
evt)
|
2017-01-18 16:28:24 +01:00
|
|
|
throw cause
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
// TODO what can this actually be? We're unitialized in the ShardRegion
|
2019-02-09 15:25:39 +01:00
|
|
|
case _ => stash()
|
2016-08-01 09:46:09 +01:00
|
|
|
}
|
2017-01-18 16:28:24 +01:00
|
|
|
|
2016-08-01 09:46:09 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object EntityRecoveryStrategy {
|
|
|
|
|
def allStrategy(): EntityRecoveryStrategy = new AllAtOnceEntityRecoveryStrategy()
|
|
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
def constantStrategy(
|
|
|
|
|
actorSystem: ActorSystem,
|
|
|
|
|
frequency: FiniteDuration,
|
|
|
|
|
numberOfEntities: Int): EntityRecoveryStrategy =
|
2016-08-01 09:46:09 +01:00
|
|
|
new ConstantRateEntityRecoveryStrategy(actorSystem, frequency, numberOfEntities)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
trait EntityRecoveryStrategy {
|
2018-12-06 14:49:47 +01:00
|
|
|
|
2016-08-01 09:46:09 +01:00
|
|
|
import ShardRegion.EntityId
|
|
|
|
|
import scala.concurrent.Future
|
|
|
|
|
|
|
|
|
|
def recoverEntities(entities: Set[EntityId]): Set[Future[Set[EntityId]]]
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final class AllAtOnceEntityRecoveryStrategy extends EntityRecoveryStrategy {
|
2018-12-06 14:49:47 +01:00
|
|
|
|
2016-08-01 09:46:09 +01:00
|
|
|
import ShardRegion.EntityId
|
2018-12-06 14:49:47 +01:00
|
|
|
|
2016-08-01 09:46:09 +01:00
|
|
|
override def recoverEntities(entities: Set[EntityId]): Set[Future[Set[EntityId]]] =
|
|
|
|
|
if (entities.isEmpty) Set.empty else Set(Future.successful(entities))
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
final class ConstantRateEntityRecoveryStrategy(
|
|
|
|
|
actorSystem: ActorSystem,
|
|
|
|
|
frequency: FiniteDuration,
|
|
|
|
|
numberOfEntities: Int)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends EntityRecoveryStrategy {
|
2018-12-06 14:49:47 +01:00
|
|
|
|
2016-08-01 09:46:09 +01:00
|
|
|
import ShardRegion.EntityId
|
|
|
|
|
import actorSystem.dispatcher
|
2017-02-21 16:17:19 +03:00
|
|
|
import akka.pattern.after
|
2016-08-01 09:46:09 +01:00
|
|
|
|
|
|
|
|
override def recoverEntities(entities: Set[EntityId]): Set[Future[Set[EntityId]]] =
|
2019-03-11 10:38:24 +01:00
|
|
|
entities
|
|
|
|
|
.grouped(numberOfEntities)
|
|
|
|
|
.foldLeft((frequency, Set[Future[Set[EntityId]]]())) {
|
|
|
|
|
case ((interval, scheduledEntityIds), entityIds) =>
|
|
|
|
|
(interval + frequency, scheduledEntityIds + scheduleEntities(interval, entityIds))
|
|
|
|
|
}
|
|
|
|
|
._2
|
2016-08-01 09:46:09 +01:00
|
|
|
|
|
|
|
|
private def scheduleEntities(interval: FiniteDuration, entityIds: Set[EntityId]) =
|
|
|
|
|
after(interval, actorSystem.scheduler)(Future.successful[Set[EntityId]](entityIds))
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|