2015-06-22 08:09:10 +02:00
|
|
|
/**
|
2017-01-04 17:37:10 +01:00
|
|
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
2015-06-22 08:09:10 +02:00
|
|
|
*/
|
|
|
|
|
package akka.cluster.sharding
|
|
|
|
|
|
|
|
|
|
import java.net.URLEncoder
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
import akka.actor.ActorLogging
|
|
|
|
|
import akka.actor.ActorRef
|
2016-08-01 09:46:09 +01:00
|
|
|
import akka.actor.ActorSystem
|
2015-06-22 08:09:10 +02:00
|
|
|
import akka.actor.Deploy
|
|
|
|
|
import akka.actor.Props
|
|
|
|
|
import akka.actor.Terminated
|
2016-08-01 09:46:09 +01:00
|
|
|
import akka.cluster.sharding.Shard.ShardCommand
|
2015-06-22 08:54:42 +02:00
|
|
|
import akka.actor.Actor
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2017-03-15 13:31:47 +01:00
|
|
|
import akka.util.MessageBufferMap
|
2016-08-01 09:46:09 +01:00
|
|
|
import scala.concurrent.Future
|
|
|
|
|
import scala.concurrent.duration.FiniteDuration
|
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._
|
|
|
|
|
import akka.actor.Stash
|
|
|
|
|
import akka.cluster.ddata.DistributedData
|
2017-04-28 18:12:30 +02:00
|
|
|
import akka.persistence.PersistentActor
|
|
|
|
|
import akka.persistence.SnapshotOffer
|
|
|
|
|
import akka.persistence.SaveSnapshotSuccess
|
2017-05-19 13:33:01 +02:00
|
|
|
import akka.persistence.DeleteSnapshotsFailure
|
2017-04-28 18:12:30 +02:00
|
|
|
import akka.persistence.DeleteMessagesSuccess
|
|
|
|
|
import akka.persistence.SaveSnapshotFailure
|
|
|
|
|
import akka.persistence.DeleteMessagesFailure
|
2017-05-19 13:33:01 +02:00
|
|
|
import akka.persistence.DeleteSnapshotsSuccess
|
2017-04-28 18:12:30 +02:00
|
|
|
import akka.persistence.SnapshotSelectionCriteria
|
|
|
|
|
import akka.persistence.RecoveryCompleted
|
2017-05-22 13:14:43 +02:00
|
|
|
import akka.actor.NoSerializationVerificationNeeded
|
2015-06-22 08:09:10 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
* @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])
|
|
|
|
|
|
|
|
|
|
@SerialVersionUID(1L) case object GetShardStats extends ShardQuery
|
|
|
|
|
|
|
|
|
|
@SerialVersionUID(1L) final case class ShardStats(shardId: ShardRegion.ShardId, entityCount: Int)
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
object State {
|
|
|
|
|
val Empty = State()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Persistent state of the Shard.
|
|
|
|
|
*/
|
2015-08-17 16:11:52 +02: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
|
|
|
*/
|
2016-06-02 14:06:57 +02:00
|
|
|
def props(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
2017-12-07 17:49:29 +01:00
|
|
|
entityProps: Props,
|
2016-06-02 14:06:57 +02:00
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
2017-01-18 16:28:24 +01:00
|
|
|
handOffStopMessage: Any,
|
2017-01-24 14:40:12 +01:00
|
|
|
replicator: ActorRef,
|
|
|
|
|
majorityMinCap: Int): Props = {
|
2017-01-18 16:28:24 +01:00
|
|
|
if (settings.rememberEntities && settings.stateStoreMode == ClusterShardingSettings.StateStoreModeDData) {
|
2017-12-07 17:49:29 +01:00
|
|
|
Props(new DDataShard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId,
|
2017-01-24 14:40:12 +01:00
|
|
|
handOffStopMessage, replicator, majorityMinCap)).withDeploy(Deploy.local)
|
2017-01-18 16:28:24 +01:00
|
|
|
} else if (settings.rememberEntities && settings.stateStoreMode == ClusterShardingSettings.StateStoreModePersistence)
|
2017-12-07 17:49:29 +01:00
|
|
|
Props(new PersistentShard(typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage))
|
2015-06-22 08:54:42 +02:00
|
|
|
.withDeploy(Deploy.local)
|
|
|
|
|
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)
|
|
|
|
|
}
|
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]]
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class Shard(
|
2016-06-02 14:06:57 +02:00
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
2017-12-07 17:49:29 +01:00
|
|
|
entityProps: Props,
|
2016-06-02 14:06:57 +02:00
|
|
|
settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
2015-06-22 08:54:42 +02:00
|
|
|
handOffStopMessage: Any) extends Actor with ActorLogging {
|
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]
|
|
|
|
|
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
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
initialized()
|
|
|
|
|
|
|
|
|
|
def initialized(): Unit = context.parent ! ShardInitialized(shardId)
|
|
|
|
|
|
2017-01-18 16:28:24 +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
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
def receiveCommand: Receive = {
|
2015-06-22 08:09:10 +02:00
|
|
|
case Terminated(ref) ⇒ receiveTerminated(ref)
|
|
|
|
|
case msg: CoordinatorMessage ⇒ receiveCoordinatorMessage(msg)
|
|
|
|
|
case msg: ShardCommand ⇒ receiveShardCommand(msg)
|
2017-05-22 10:08:18 +02:00
|
|
|
case msg: ShardRegion.StartEntity ⇒ receiveStartEntity(msg)
|
|
|
|
|
case msg: ShardRegion.StartEntityAck ⇒ receiveStartEntityAck(msg)
|
2015-06-22 08:09:10 +02:00
|
|
|
case msg: ShardRegionCommand ⇒ receiveShardRegionCommand(msg)
|
2015-09-23 11:24:39 +02:00
|
|
|
case msg: ShardQuery ⇒ receiveShardQuery(msg)
|
2015-06-22 08:09:10 +02:00
|
|
|
case msg if extractEntityId.isDefinedAt(msg) ⇒ deliverMessage(msg, sender())
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receiveShardCommand(msg: ShardCommand): Unit = msg match {
|
2016-08-01 09:46:09 +01:00
|
|
|
case RestartEntity(id) ⇒ getEntity(id)
|
2017-05-22 10:08:18 +02:00
|
|
|
case RestartEntities(ids) ⇒ restartEntities(ids)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receiveStartEntity(start: ShardRegion.StartEntity): Unit = {
|
|
|
|
|
log.debug("Got a request from [{}] to start entity [{}] in shard [{}]", sender(), start.entityId, shardId)
|
|
|
|
|
getEntity(start.entityId)
|
|
|
|
|
sender() ! ShardRegion.StartEntityAck(start.entityId, shardId)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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)
|
|
|
|
|
processChange(EntityStopped(ack.entityId)) { _ ⇒
|
|
|
|
|
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 {
|
|
|
|
|
case Passivate(stopMessage) ⇒ passivate(sender(), stopMessage)
|
|
|
|
|
case _ ⇒ unhandled(msg)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receiveCoordinatorMessage(msg: CoordinatorMessage): Unit = msg match {
|
|
|
|
|
case HandOff(`shardId`) ⇒ handOff(sender())
|
|
|
|
|
case HandOff(shard) ⇒ log.warning("Shard [{}] can not hand off for another Shard [{}]", shardId, shard)
|
|
|
|
|
case _ ⇒ unhandled(msg)
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-23 11:24:39 +02:00
|
|
|
def receiveShardQuery(msg: ShardQuery): Unit = msg match {
|
|
|
|
|
case GetCurrentShardState ⇒ sender() ! CurrentShardState(shardId, refById.keySet)
|
|
|
|
|
case GetShardStats ⇒ sender() ! ShardStats(shardId, state.entities.size)
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-22 08:09:10 +02:00
|
|
|
def handOff(replyTo: ActorRef): Unit = handOffStopper match {
|
|
|
|
|
case Some(_) ⇒ log.warning("HandOff shard [{}] received during existing handOff", shardId)
|
|
|
|
|
case None ⇒
|
|
|
|
|
log.debug("HandOff shard [{}]", shardId)
|
|
|
|
|
|
|
|
|
|
if (state.entities.nonEmpty) {
|
|
|
|
|
handOffStopper = Some(context.watch(context.actorOf(
|
|
|
|
|
handOffStopperProps(shardId, replyTo, idByRef.keySet, handOffStopMessage))))
|
|
|
|
|
|
|
|
|
|
//During hand off we only care about watching for termination of the hand off stopper
|
|
|
|
|
context become {
|
|
|
|
|
case Terminated(ref) ⇒ receiveTerminated(ref)
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
replyTo ! ShardStopped(shardId)
|
|
|
|
|
context stop self
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receiveTerminated(ref: ActorRef): Unit = {
|
2017-02-21 16:17:19 +03:00
|
|
|
if (handOffStopper.contains(ref))
|
2015-06-22 08:09:10 +02: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
|
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 {
|
2017-03-10 14:21:37 +01:00
|
|
|
case Some(id) ⇒ if (!messageBuffers.contains(id)) {
|
2015-06-22 08:09:10 +02:00
|
|
|
log.debug("Passivating started on entity {}", id)
|
|
|
|
|
|
|
|
|
|
passivating = passivating + entity
|
2017-03-15 13:31:47 +01:00
|
|
|
messageBuffers.add(id)
|
2015-06-22 08:09:10 +02:00
|
|
|
entity ! stopMessage
|
2017-02-21 14:08:43 +01:00
|
|
|
} else {
|
|
|
|
|
log.debug("Passivation already in progress for {}. Not sending stopMessage back to entity.", entity)
|
|
|
|
|
}
|
2017-03-10 14:21:37 +01:00
|
|
|
case None ⇒ log.debug("Unknown entity {}. Not sending stopMessage back to entity.", entity)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
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)
|
|
|
|
|
getEntity(event.entityId)
|
|
|
|
|
|
|
|
|
|
//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 {
|
2015-06-22 08:09:10 +02:00
|
|
|
case (msg, snd) ⇒ deliverMessage(msg, snd)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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 {
|
2017-03-15 13:31:47 +01:00
|
|
|
messageBuffers.contains(id) match {
|
|
|
|
|
case false ⇒ deliverTo(id, msg, payload, snd)
|
2015-06-22 08:09:10 +02:00
|
|
|
|
2017-03-15 13:31:47 +01:00
|
|
|
case true if messageBuffers.totalSize >= bufferSize ⇒
|
2015-06-22 08:09:10 +02:00
|
|
|
log.debug("Buffer is full, dropping message for entity [{}]", id)
|
|
|
|
|
context.system.deadLetters ! msg
|
|
|
|
|
|
2017-03-15 13:31:47 +01:00
|
|
|
case true ⇒
|
2015-06-22 08:09:10 +02:00
|
|
|
log.debug("Message for entity [{}] buffered", id)
|
2017-03-15 13:31:47 +01:00
|
|
|
messageBuffers.append(id, msg, snd)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def deliverTo(id: EntityId, msg: Any, payload: Msg, snd: ActorRef): Unit = {
|
|
|
|
|
val name = URLEncoder.encode(id, "utf-8")
|
|
|
|
|
context.child(name) match {
|
2015-06-22 08:54:42 +02:00
|
|
|
case Some(actor) ⇒ actor.tell(payload, snd)
|
|
|
|
|
case None ⇒ getEntity(id).tell(payload, snd)
|
2015-06-22 08:09:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def getEntity(id: EntityId): ActorRef = {
|
|
|
|
|
val name = URLEncoder.encode(id, "utf-8")
|
|
|
|
|
context.child(name).getOrElse {
|
|
|
|
|
log.debug("Starting entity [{}] in shard [{}]", id, shardId)
|
|
|
|
|
|
|
|
|
|
val a = context.watch(context.actorOf(entityProps, name))
|
|
|
|
|
idByRef = idByRef.updated(a, id)
|
|
|
|
|
refById = refById.updated(id, a)
|
|
|
|
|
state = state.copy(state.entities + id)
|
|
|
|
|
a
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-06-22 08:54:42 +02:00
|
|
|
|
2017-05-22 10:08:18 +02:00
|
|
|
private[akka] object RememberEntityStarter {
|
|
|
|
|
def props(
|
2017-05-22 13:14:43 +02:00
|
|
|
region: ActorRef,
|
2017-05-22 10:08:18 +02:00
|
|
|
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
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class RememberEntityStarter(
|
2017-05-22 13:14:43 +02:00
|
|
|
region: ActorRef,
|
2017-05-22 10:08:18 +02:00
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
|
|
|
|
ids: Set[ShardRegion.EntityId],
|
|
|
|
|
settings: ClusterShardingSettings,
|
2017-05-22 13:14:43 +02:00
|
|
|
requestor: ActorRef) extends Actor with ActorLogging {
|
2017-05-22 10:08:18 +02:00
|
|
|
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
import scala.concurrent.duration._
|
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 = {
|
|
|
|
|
ids.foreach(id ⇒ region ! ShardRegion.StartEntity(id))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def receive = {
|
|
|
|
|
case ack: ShardRegion.StartEntityAck ⇒
|
|
|
|
|
waitingForAck -= ack.entityId
|
|
|
|
|
// inform whoever requested the start that it happened
|
|
|
|
|
requestor ! ack
|
|
|
|
|
if (waitingForAck.isEmpty) context.stop(self)
|
|
|
|
|
|
|
|
|
|
case Tick ⇒
|
|
|
|
|
sendStart(waitingForAck)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
tickTask.cancel()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API: Common things for PersistentShard and DDataShard
|
|
|
|
|
*/
|
|
|
|
|
private[akka] trait RememberingShard { selfType: Shard ⇒
|
|
|
|
|
import ShardRegion.{ EntityId, Msg }
|
|
|
|
|
import Shard._
|
|
|
|
|
import akka.pattern.pipe
|
|
|
|
|
|
|
|
|
|
protected val settings: ClusterShardingSettings
|
|
|
|
|
|
|
|
|
|
protected val rememberedEntitiesRecoveryStrategy: EntityRecoveryStrategy = {
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
entityRecoveryStrategy match {
|
|
|
|
|
case "all" ⇒ EntityRecoveryStrategy.allStrategy()
|
|
|
|
|
case "constant" ⇒ EntityRecoveryStrategy.constantStrategy(
|
|
|
|
|
context.system,
|
|
|
|
|
entityRecoveryConstantRateStrategyFrequency,
|
|
|
|
|
entityRecoveryConstantRateStrategyNumberOfEntities)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
protected def restartRememberedEntities(): Unit = {
|
|
|
|
|
rememberedEntitiesRecoveryStrategy.recoverEntities(state.entities).foreach { scheduledRecovery ⇒
|
|
|
|
|
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
|
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 {
|
|
|
|
|
case Some(actor) ⇒
|
|
|
|
|
actor.tell(payload, snd)
|
|
|
|
|
|
|
|
|
|
case None ⇒
|
|
|
|
|
//Note; we only do this if remembering, otherwise the buffer is an overhead
|
2017-03-15 13:31:47 +01:00
|
|
|
messageBuffers.append(id, msg, snd)
|
2017-01-18 16:28:24 +01:00
|
|
|
processChange(EntityStarted(id))(sendMsgBuffer)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
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]]
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class PersistentShard(
|
2017-01-18 16:28:24 +01:00
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
2017-12-07 17:49:29 +01:00
|
|
|
entityProps: Props,
|
2017-01-18 16:28:24 +01:00
|
|
|
override val settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
|
|
|
|
handOffStopMessage: Any) extends Shard(
|
2017-12-07 17:49:29 +01:00
|
|
|
typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage)
|
2017-01-18 16:28:24 +01:00
|
|
|
with RememberingShard with PersistentActor with ActorLogging {
|
2015-06-22 08:54:42 +02:00
|
|
|
|
|
|
|
|
import ShardRegion.{ EntityId, Msg }
|
2017-01-18 16:28:24 +01:00
|
|
|
import Shard._
|
2015-06-22 08:54:42 +02:00
|
|
|
import settings.tuningParameters._
|
|
|
|
|
|
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
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
// would be initialized after recovery completed
|
|
|
|
|
override def initialized(): Unit = {}
|
|
|
|
|
|
2015-06-22 08:54:42 +02:00
|
|
|
override def receive = receiveCommand
|
|
|
|
|
|
2017-01-18 16:28:24 +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 = {
|
|
|
|
|
case EntityStarted(id) ⇒ state = state.copy(state.entities + id)
|
|
|
|
|
case EntityStopped(id) ⇒ state = state.copy(state.entities - id)
|
|
|
|
|
case SnapshotOffer(_, snapshot: State) ⇒ state = snapshot
|
2015-08-20 13:24:39 +03:00
|
|
|
case RecoveryCompleted ⇒
|
2016-08-01 09:46:09 +01:00
|
|
|
restartRememberedEntities()
|
2015-08-20 13:24:39 +03:00
|
|
|
super.initialized()
|
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
|
|
|
}
|
|
|
|
|
|
2016-03-15 15:35:25 +01:00
|
|
|
override def receiveCommand: Receive = ({
|
2017-02-21 16:17:19 +03:00
|
|
|
case SaveSnapshotSuccess(m) ⇒
|
2016-03-15 15:35:25 +01:00
|
|
|
log.debug("PersistentShard snapshot saved successfully")
|
2017-02-21 16:17:19 +03:00
|
|
|
/*
|
|
|
|
|
* delete old events but keep the latest around because
|
|
|
|
|
*
|
|
|
|
|
* it's not safe to delete all events immediate because snapshots are typically stored with a weaker consistency
|
|
|
|
|
* level which means that a replay might "see" the deleted events before it sees the stored snapshot,
|
|
|
|
|
* i.e. it will use an older snapshot and then not replay the full sequence of events
|
|
|
|
|
*
|
|
|
|
|
* for debugging if something goes wrong in production it's very useful to be able to inspect the events
|
|
|
|
|
*/
|
|
|
|
|
val deleteToSequenceNr = m.sequenceNr - keepNrOfBatches * snapshotAfter
|
|
|
|
|
if (deleteToSequenceNr > 0) {
|
|
|
|
|
deleteMessages(deleteToSequenceNr)
|
|
|
|
|
}
|
|
|
|
|
|
2016-03-15 15:35:25 +01:00
|
|
|
case SaveSnapshotFailure(_, reason) ⇒
|
|
|
|
|
log.warning("PersistentShard snapshot failure: {}", reason.getMessage)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
|
|
|
|
case DeleteMessagesSuccess(toSequenceNr) ⇒
|
|
|
|
|
log.debug("PersistentShard messages to {} deleted successfully", toSequenceNr)
|
|
|
|
|
deleteSnapshots(SnapshotSelectionCriteria(maxSequenceNr = toSequenceNr - 1))
|
|
|
|
|
|
|
|
|
|
case DeleteMessagesFailure(reason, toSequenceNr) ⇒
|
|
|
|
|
log.warning("PersistentShard messages to {} deletion failure: {}", toSequenceNr, reason.getMessage)
|
|
|
|
|
|
2017-05-19 13:33:01 +02:00
|
|
|
case DeleteSnapshotsSuccess(m) ⇒
|
2017-02-21 16:17:19 +03:00
|
|
|
log.debug("PersistentShard snapshots matching {} deleted successfully", m)
|
|
|
|
|
|
2017-05-19 13:33:01 +02:00
|
|
|
case DeleteSnapshotsFailure(m, reason) ⇒
|
|
|
|
|
log.warning("PersistentShard snapshots matching {} deletion failure: {}", m, reason.getMessage)
|
2017-02-21 16:17:19 +03:00
|
|
|
|
2016-03-15 15:35:25 +01:00
|
|
|
}: Receive).orElse(super.receiveCommand)
|
|
|
|
|
|
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]]
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class DDataShard(
|
|
|
|
|
typeName: String,
|
|
|
|
|
shardId: ShardRegion.ShardId,
|
2017-12-07 17:49:29 +01:00
|
|
|
entityProps: Props,
|
2017-01-18 16:28:24 +01:00
|
|
|
override val settings: ClusterShardingSettings,
|
|
|
|
|
extractEntityId: ShardRegion.ExtractEntityId,
|
|
|
|
|
extractShardId: ShardRegion.ExtractShardId,
|
|
|
|
|
handOffStopMessage: Any,
|
2017-01-24 14:40:12 +01:00
|
|
|
replicator: ActorRef,
|
|
|
|
|
majorityMinCap: Int) extends Shard(
|
2017-12-07 17:49:29 +01:00
|
|
|
typeName, shardId, entityProps, settings, extractEntityId, extractShardId, handOffStopMessage)
|
2017-01-18 16:28:24 +01:00
|
|
|
with RememberingShard with Stash with ActorLogging {
|
|
|
|
|
|
|
|
|
|
import ShardRegion.{ EntityId, Msg }
|
|
|
|
|
import Shard._
|
|
|
|
|
import settings.tuningParameters._
|
|
|
|
|
|
2017-01-24 14:40:12 +01:00
|
|
|
private val readMajority = ReadMajority(
|
|
|
|
|
settings.tuningParameters.waitingForStateTimeout,
|
|
|
|
|
majorityMinCap)
|
|
|
|
|
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]] =
|
|
|
|
|
Array.tabulate(numberOfKeys)(i ⇒ ORSetKey[EntityId](s"shard-${typeName}-${shardId}-$i"))
|
|
|
|
|
|
|
|
|
|
private def key(entityId: EntityId): ORSetKey[EntityId] = {
|
|
|
|
|
val i = (math.abs(entityId.hashCode) % numberOfKeys)
|
|
|
|
|
stateKeys(i)
|
2015-06-22 08:54:42 +02:00
|
|
|
}
|
|
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
// get initial state from ddata replicator
|
|
|
|
|
getState()
|
2015-06-22 08:54:42 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
private def getState(): Unit = {
|
|
|
|
|
(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
|
|
|
// would be initialized after recovery completed
|
|
|
|
|
override def initialized(): Unit = {}
|
|
|
|
|
|
|
|
|
|
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
|
|
|
|
|
if (newGotKeys.size == numberOfKeys)
|
|
|
|
|
recoveryCompleted()
|
|
|
|
|
else
|
|
|
|
|
context.become(waitingForState(newGotKeys))
|
2016-08-01 09:46:09 +01:00
|
|
|
}
|
2017-01-18 16:28:24 +01:00
|
|
|
|
|
|
|
|
{
|
|
|
|
|
case g @ GetSuccess(_, Some(i: Int)) ⇒
|
|
|
|
|
val key = stateKeys(i)
|
|
|
|
|
state = state.copy(entities = state.entities union (g.get(key).elements))
|
|
|
|
|
receiveOne(i)
|
|
|
|
|
|
|
|
|
|
case GetFailure(_, _) ⇒
|
|
|
|
|
log.error(
|
|
|
|
|
"The DDataShard was unable to get an initial state within 'waiting-for-state-timeout': {} millis",
|
|
|
|
|
waitingForStateTimeout.toMillis)
|
|
|
|
|
// parent ShardRegion supervisor will notice that it terminated and will start it again, after backoff
|
|
|
|
|
context.stop(self)
|
|
|
|
|
|
|
|
|
|
case NotFound(_, Some(i: Int)) ⇒
|
|
|
|
|
receiveOne(i)
|
|
|
|
|
|
|
|
|
|
case _ ⇒
|
|
|
|
|
stash()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def recoveryCompleted(): Unit = {
|
|
|
|
|
restartRememberedEntities()
|
|
|
|
|
super.initialized()
|
|
|
|
|
log.debug("DDataShard recovery completed shard [{}] with [{}] entities", shardId, state.entities.size)
|
|
|
|
|
unstashAll()
|
|
|
|
|
context.become(receiveCommand)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def processChange[E <: StateChange](event: E)(handler: E ⇒ Unit): Unit = {
|
|
|
|
|
context.become(waitingForUpdate(event, handler), discardOld = false)
|
|
|
|
|
sendUpdate(event, retryCount = 1)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def sendUpdate(evt: StateChange, retryCount: Int) = {
|
2017-01-24 14:40:12 +01:00
|
|
|
replicator ! Update(key(evt.entityId), ORSet.empty[EntityId], writeMajority,
|
2017-01-18 16:28:24 +01:00
|
|
|
Some((evt, retryCount))) { existing ⇒
|
|
|
|
|
evt match {
|
|
|
|
|
case EntityStarted(id) ⇒ existing + id
|
|
|
|
|
case EntityStopped(id) ⇒ existing - id
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// this state will stash all messages until it receives UpdateSuccess
|
|
|
|
|
private def waitingForUpdate[E <: StateChange](evt: E, afterUpdateCallback: E ⇒ Unit): Receive = {
|
|
|
|
|
case UpdateSuccess(_, Some((`evt`, _))) ⇒
|
|
|
|
|
log.debug("The DDataShard state was successfully updated with {}", evt)
|
|
|
|
|
context.unbecome()
|
|
|
|
|
afterUpdateCallback(evt)
|
|
|
|
|
unstashAll()
|
|
|
|
|
|
|
|
|
|
case UpdateTimeout(_, Some((`evt`, retryCount: Int))) ⇒
|
|
|
|
|
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={}. " +
|
|
|
|
|
"Shard will be restarted after backoff.",
|
|
|
|
|
maxUpdateAttempts, updatingStateTimeout.toMillis, evt)
|
|
|
|
|
context.stop(self)
|
|
|
|
|
} else {
|
|
|
|
|
log.warning(
|
|
|
|
|
"The DDataShard was unable to update state, attempt {} of {}, within 'updating-state-timeout'={} millis, event={}",
|
|
|
|
|
retryCount, maxUpdateAttempts, updatingStateTimeout.toMillis, evt)
|
|
|
|
|
sendUpdate(evt, retryCount + 1)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case ModifyFailure(_, error, cause, Some((`evt`, _))) ⇒
|
|
|
|
|
log.error(
|
|
|
|
|
cause,
|
|
|
|
|
"The DDataShard was unable to update state with error {} and event {}. Shard will be restarted",
|
|
|
|
|
error,
|
|
|
|
|
evt)
|
|
|
|
|
throw cause
|
|
|
|
|
|
|
|
|
|
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()
|
|
|
|
|
|
|
|
|
|
def constantStrategy(actorSystem: ActorSystem, frequency: FiniteDuration, numberOfEntities: Int): EntityRecoveryStrategy =
|
|
|
|
|
new ConstantRateEntityRecoveryStrategy(actorSystem, frequency, numberOfEntities)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
trait EntityRecoveryStrategy {
|
|
|
|
|
import ShardRegion.EntityId
|
|
|
|
|
import scala.concurrent.Future
|
|
|
|
|
|
|
|
|
|
def recoverEntities(entities: Set[EntityId]): Set[Future[Set[EntityId]]]
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final class AllAtOnceEntityRecoveryStrategy extends EntityRecoveryStrategy {
|
|
|
|
|
import ShardRegion.EntityId
|
|
|
|
|
override def recoverEntities(entities: Set[EntityId]): Set[Future[Set[EntityId]]] =
|
|
|
|
|
if (entities.isEmpty) Set.empty else Set(Future.successful(entities))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final class ConstantRateEntityRecoveryStrategy(actorSystem: ActorSystem, frequency: FiniteDuration, numberOfEntities: Int) extends EntityRecoveryStrategy {
|
|
|
|
|
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]]] =
|
|
|
|
|
entities.grouped(numberOfEntities).foldLeft((frequency, Set[Future[Set[EntityId]]]())) {
|
|
|
|
|
case ((interval, scheduledEntityIds), entityIds) ⇒
|
|
|
|
|
(interval + frequency, scheduledEntityIds + scheduleEntities(interval, entityIds))
|
|
|
|
|
}._2
|
|
|
|
|
|
|
|
|
|
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
|
|
|
}
|