replace unicode arrows

* ⇒, →, ←
* because we don't want to show them in documentation snippets and
  then it's complicated to avoid that when snippets are
  located in src/test/scala in individual modules
* dont replace object `→` in FSM.scala and PersistentFSM.scala
This commit is contained in:
Patrik Nordwall 2019-02-09 15:25:39 +01:00
parent e4d38f92a4
commit 5c96a5f556
1521 changed files with 18846 additions and 18786 deletions

View file

@ -98,7 +98,7 @@ private[akka] object Shard {
def props(
typeName: String,
shardId: ShardRegion.ShardId,
entityProps: String Props,
entityProps: String => Props,
settings: ClusterShardingSettings,
extractEntityId: ShardRegion.ExtractEntityId,
extractShardId: ShardRegion.ExtractShardId,
@ -131,7 +131,7 @@ private[akka] object Shard {
private[akka] class Shard(
typeName: String,
shardId: ShardRegion.ShardId,
entityProps: String Props,
entityProps: String => Props,
settings: ClusterShardingSettings,
extractEntityId: ShardRegion.ExtractEntityId,
extractShardId: ShardRegion.ExtractShardId,
@ -165,26 +165,26 @@ private[akka] class Shard(
def initialized(): Unit = context.parent ! ShardInitialized(shardId)
def processChange[E <: StateChange](event: E)(handler: E Unit): Unit =
def processChange[E <: StateChange](event: E)(handler: E => Unit): Unit =
handler(event)
def receive = receiveCommand
def receiveCommand: Receive = {
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()
case msg if extractEntityId.isDefinedAt(msg) deliverMessage(msg, sender())
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()
case msg if extractEntityId.isDefinedAt(msg) => deliverMessage(msg, sender())
}
def receiveShardCommand(msg: ShardCommand): Unit = msg match {
case RestartEntity(id) getOrCreateEntity(id)
case RestartEntities(ids) restartEntities(ids)
case RestartEntity(id) => getOrCreateEntity(id)
case RestartEntities(ids) => restartEntities(ids)
}
def receiveStartEntity(start: ShardRegion.StartEntity): Unit = {
@ -193,13 +193,13 @@ private[akka] class Shard(
if (passivateIdleTask.isDefined) {
lastMessageTimestamp = lastMessageTimestamp.updated(start.entityId, System.nanoTime())
}
getOrCreateEntity(start.entityId, _ processChange(EntityStarted(start.entityId))(_ requester ! ShardRegion.StartEntityAck(start.entityId, shardId)))
getOrCreateEntity(start.entityId, _ => processChange(EntityStarted(start.entityId))(_ => requester ! 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)) { _
processChange(EntityStopped(ack.entityId)) { _ =>
state = state.copy(state.entities - ack.entityId)
messageBuffers.remove(ack.entityId)
}
@ -211,24 +211,24 @@ private[akka] class Shard(
}
def receiveShardRegionCommand(msg: ShardRegionCommand): Unit = msg match {
case Passivate(stopMessage) passivate(sender(), stopMessage)
case _ unhandled(msg)
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)
case HandOff(`shardId`) => handOff(sender())
case HandOff(shard) => log.warning("Shard [{}] can not hand off for another Shard [{}]", shardId, shard)
case _ => unhandled(msg)
}
def receiveShardQuery(msg: ShardQuery): Unit = msg match {
case GetCurrentShardState sender() ! CurrentShardState(shardId, refById.keySet)
case GetShardStats sender() ! ShardStats(shardId, state.entities.size)
case GetCurrentShardState => sender() ! CurrentShardState(shardId, refById.keySet)
case GetShardStats => sender() ! ShardStats(shardId, state.entities.size)
}
def handOff(replyTo: ActorRef): Unit = handOffStopper match {
case Some(_) log.warning("HandOff shard [{}] received during existing handOff", shardId)
case None
case Some(_) => log.warning("HandOff shard [{}] received during existing handOff", shardId)
case None =>
log.debug("HandOff shard [{}]", shardId)
if (state.entities.nonEmpty) {
@ -238,7 +238,7 @@ private[akka] class Shard(
//During hand off we only care about watching for termination of the hand off stopper
context become {
case Terminated(ref) receiveTerminated(ref)
case Terminated(ref) => receiveTerminated(ref)
}
} else {
replyTo ! ShardStopped(shardId)
@ -272,21 +272,21 @@ private[akka] class Shard(
def passivate(entity: ActorRef, stopMessage: Any): Unit = {
idByRef.get(entity) match {
case Some(id) if (!messageBuffers.contains(id)) {
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)
}
case None log.debug("Unknown entity {}. Not sending stopMessage back to entity.", entity)
case None => log.debug("Unknown entity {}. Not sending stopMessage back to entity.", entity)
}
}
def passivateIdleEntities(): Unit = {
val deadline = System.nanoTime() - settings.passivateIdleEntityAfter.toNanos
val refsToPassivate = lastMessageTimestamp.collect {
case (entityId, lastMessageTimestamp) if lastMessageTimestamp < deadline refById(entityId)
case (entityId, lastMessageTimestamp) if lastMessageTimestamp < deadline => refById(entityId)
}
if (refsToPassivate.nonEmpty) {
log.debug("Passivating [{}] idle entities", refsToPassivate.size)
@ -313,7 +313,7 @@ private[akka] class Shard(
//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)
case (msg, snd) => deliverMessage(msg, snd)
}
}
}
@ -325,18 +325,18 @@ private[akka] class Shard(
context.system.deadLetters ! msg
} else {
payload match {
case start: ShardRegion.StartEntity
case start: ShardRegion.StartEntity =>
// in case it was wrapped, used in Typed
receiveStartEntity(start)
case _
case _ =>
messageBuffers.contains(id) match {
case false deliverTo(id, msg, payload, snd)
case false => deliverTo(id, msg, payload, snd)
case true if messageBuffers.totalSize >= bufferSize
case true if messageBuffers.totalSize >= bufferSize =>
log.debug("Buffer is full, dropping message for entity [{}]", id)
context.system.deadLetters ! msg
case true
case true =>
log.debug("Message for entity [{}] buffered", id)
messageBuffers.append(id, msg, snd)
}
@ -351,11 +351,11 @@ private[akka] class Shard(
getOrCreateEntity(id).tell(payload, snd)
}
def getOrCreateEntity(id: EntityId, onCreate: ActorRef Unit = ConstantFun.scalaAnyToUnit): ActorRef = {
def getOrCreateEntity(id: EntityId, onCreate: ActorRef => Unit = ConstantFun.scalaAnyToUnit): ActorRef = {
val name = URLEncoder.encode(id, "utf-8")
context.child(name) match {
case Some(child) child
case None
case Some(child) => child
case None =>
log.debug("Starting entity [{}] in shard [{}]", id, shardId)
val a = context.watch(context.actorOf(entityProps(id), name))
idByRef = idByRef.updated(a, id)
@ -413,17 +413,17 @@ private[akka] class RememberEntityStarter(
def sendStart(ids: Set[ShardRegion.EntityId]): Unit = {
// these go through the region rather the directly to the shard
// so that shard mapping changes are picked up
ids.foreach(id region ! ShardRegion.StartEntity(id))
ids.foreach(id => region ! ShardRegion.StartEntity(id))
}
override def receive: Receive = {
case ack: ShardRegion.StartEntityAck
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
case Tick =>
sendStart(waitingForAck)
}
@ -437,7 +437,7 @@ private[akka] class RememberEntityStarter(
* INTERNAL API: Common things for PersistentShard and DDataShard
*/
private[akka] trait RememberingShard {
selfType: Shard
selfType: Shard =>
import ShardRegion.{ EntityId, Msg }
import Shard._
@ -448,8 +448,8 @@ private[akka] trait RememberingShard {
protected val rememberedEntitiesRecoveryStrategy: EntityRecoveryStrategy = {
import settings.tuningParameters._
entityRecoveryStrategy match {
case "all" EntityRecoveryStrategy.allStrategy()
case "constant" EntityRecoveryStrategy.constantStrategy(
case "all" => EntityRecoveryStrategy.allStrategy()
case "constant" => EntityRecoveryStrategy.constantStrategy(
context.system,
entityRecoveryConstantRateStrategyFrequency,
entityRecoveryConstantRateStrategyNumberOfEntities)
@ -457,7 +457,7 @@ private[akka] trait RememberingShard {
}
protected def restartRememberedEntities(): Unit = {
rememberedEntitiesRecoveryStrategy.recoverEntities(state.entities).foreach { scheduledRecovery
rememberedEntitiesRecoveryStrategy.recoverEntities(state.entities).foreach { scheduledRecovery =>
import context.dispatcher
scheduledRecovery.filter(_.nonEmpty).map(RestartEntities).pipeTo(self)
}
@ -490,9 +490,9 @@ private[akka] trait RememberingShard {
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)
case Some(actor) =>
actor.tell(payload, snd)
case None
case None =>
if (state.entities.contains(id)) {
require(!messageBuffers.contains(id), s"Message buffers contains id [$id].")
getOrCreateEntity(id).tell(payload, snd)
@ -517,7 +517,7 @@ private[akka] trait RememberingShard {
private[akka] class PersistentShard(
typeName: String,
shardId: ShardRegion.ShardId,
entityProps: String Props,
entityProps: String => Props,
override val settings: ClusterShardingSettings,
extractEntityId: ShardRegion.ExtractEntityId,
extractShardId: ShardRegion.ExtractShardId,
@ -539,7 +539,7 @@ private[akka] class PersistentShard(
override def receive = receiveCommand
override def processChange[E <: StateChange](event: E)(handler: E Unit): Unit = {
override def processChange[E <: StateChange](event: E)(handler: E => Unit): Unit = {
saveSnapshotWhenNeeded()
persist(event)(handler)
}
@ -552,24 +552,24 @@ private[akka] class PersistentShard(
}
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
case RecoveryCompleted
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 =>
restartRememberedEntities()
super.initialized()
log.debug("PersistentShard recovery completed shard [{}] with [{}] entities", shardId, state.entities.size)
}
override def receiveCommand: Receive = ({
case e: SaveSnapshotSuccess
case e: SaveSnapshotSuccess =>
log.debug("PersistentShard snapshot saved successfully")
internalDeleteMessagesBeforeSnapshot(e, keepNrOfBatches, snapshotAfter)
case SaveSnapshotFailure(_, reason)
case SaveSnapshotFailure(_, reason) =>
log.warning("PersistentShard snapshot failure: [{}]", reason.getMessage)
case DeleteMessagesSuccess(toSequenceNr)
case DeleteMessagesSuccess(toSequenceNr) =>
val deleteTo = toSequenceNr - 1
val deleteFrom = math.max(0, deleteTo - (keepNrOfBatches * snapshotAfter))
log.debug("PersistentShard messages to [{}] deleted successfully. Deleting snapshots from [{}] to [{}]", toSequenceNr, deleteFrom, deleteTo)
@ -578,13 +578,13 @@ private[akka] class PersistentShard(
maxSequenceNr = deleteTo
))
case DeleteMessagesFailure(reason, toSequenceNr)
case DeleteMessagesFailure(reason, toSequenceNr) =>
log.warning("PersistentShard messages to [{}] deletion failure: [{}]", toSequenceNr, reason.getMessage)
case DeleteSnapshotsSuccess(m)
case DeleteSnapshotsSuccess(m) =>
log.debug("PersistentShard snapshots matching [{}] deleted successfully", m)
case DeleteSnapshotsFailure(m, reason)
case DeleteSnapshotsFailure(m, reason) =>
log.warning("PersistentShard snapshots matching [{}] deletion failure: [{}]", m, reason.getMessage)
}: Receive).orElse(super.receiveCommand)
@ -603,7 +603,7 @@ private[akka] class PersistentShard(
private[akka] class DDataShard(
typeName: String,
shardId: ShardRegion.ShardId,
entityProps: String Props,
entityProps: String => Props,
override val settings: ClusterShardingSettings,
extractEntityId: ShardRegion.ExtractEntityId,
extractShardId: ShardRegion.ExtractShardId,
@ -633,7 +633,7 @@ private[akka] class DDataShard(
// 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"))
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))
@ -644,7 +644,7 @@ private[akka] class DDataShard(
getState()
private def getState(): Unit = {
(0 until numberOfKeys).map { i
(0 until numberOfKeys).map { i =>
replicator ! Get(stateKeys(i), readMajority, Some(i))
}
}
@ -665,22 +665,22 @@ private[akka] class DDataShard(
}
{
case g @ GetSuccess(_, Some(i: Int))
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(_, _)
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))
case NotFound(_, Some(i: Int)) =>
receiveOne(i)
case _
case _ =>
stash()
}
}
@ -693,30 +693,30 @@ private[akka] class DDataShard(
context.become(receiveCommand)
}
override def processChange[E <: StateChange](event: E)(handler: E Unit): Unit = {
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) = {
replicator ! Update(key(evt.entityId), ORSet.empty[EntityId], writeMajority,
Some((evt, retryCount))) { existing
Some((evt, retryCount))) { existing =>
evt match {
case EntityStarted(id) existing + id
case EntityStopped(id) existing - id
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`, _)))
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)))
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(
@ -731,7 +731,7 @@ private[akka] class DDataShard(
sendUpdate(evt, retryCount + 1)
}
case ModifyFailure(_, error, cause, Some((`evt`, _)))
case ModifyFailure(_, error, cause, Some((`evt`, _))) =>
log.error(
cause,
"The DDataShard was unable to update state with error {} and event {}. Shard will be restarted",
@ -739,7 +739,7 @@ private[akka] class DDataShard(
evt)
throw cause
case _ stash()
case _ => stash()
}
}
@ -775,7 +775,7 @@ final class ConstantRateEntityRecoveryStrategy(actorSystem: ActorSystem, frequen
override def recoverEntities(entities: Set[EntityId]): Set[Future[Set[EntityId]]] =
entities.grouped(numberOfEntities).foldLeft((frequency, Set[Future[Set[EntityId]]]())) {
case ((interval, scheduledEntityIds), entityIds)
case ((interval, scheduledEntityIds), entityIds) =>
(interval + frequency, scheduledEntityIds + scheduleEntities(interval, entityIds))
}._2