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

@ -33,7 +33,7 @@ import akka.cluster.UniqueAddress
import akka.serialization.SerializationExtension
import akka.util.ByteString
import com.typesafe.config.Config
import java.util.function.{ Function JFunction }
import java.util.function.{ Function => JFunction }
import akka.dispatch.Dispatchers
import akka.actor.DeadLetterSuppression
import akka.cluster.ddata.Key.KeyR
@ -70,13 +70,13 @@ object ReplicatorSettings {
*/
def apply(config: Config): ReplicatorSettings = {
val dispatcher = config.getString("use-dispatcher") match {
case "" Dispatchers.DefaultDispatcherId
case id id
case "" => Dispatchers.DefaultDispatcherId
case id => id
}
val pruningInterval = toRootLowerCase(config.getString("pruning-interval")) match {
case "off" | "false" Duration.Zero
case _ config.getDuration("pruning-interval", MILLISECONDS).millis
case "off" | "false" => Duration.Zero
case _ => config.getDuration("pruning-interval", MILLISECONDS).millis
}
import scala.collection.JavaConverters._
@ -108,7 +108,7 @@ object ReplicatorSettings {
*/
@InternalApi private[akka] def name(system: ActorSystem, modifier: Option[String]): String = {
val name = system.settings.config.getString("akka.cluster.distributed-data.name")
modifier.map(s s + name.take(1).toUpperCase + name.drop(1)).getOrElse(name)
modifier.map(s => s + name.take(1).toUpperCase + name.drop(1)).getOrElse(name)
}
}
@ -221,8 +221,8 @@ final class ReplicatorSettings(
def withDispatcher(dispatcher: String): ReplicatorSettings = {
val d = dispatcher match {
case "" Dispatchers.DefaultDispatcherId
case id id
case "" => Dispatchers.DefaultDispatcherId
case id => id
}
copy(dispatcher = d)
}
@ -504,12 +504,12 @@ object Replicator {
*/
def apply[A <: ReplicatedData](
key: Key[A], initial: A, writeConsistency: WriteConsistency,
request: Option[Any] = None)(modify: A A): Update[A] =
request: Option[Any] = None)(modify: A => A): Update[A] =
Update(key, writeConsistency, request)(modifyWithInitial(initial, modify))
private def modifyWithInitial[A <: ReplicatedData](initial: A, modify: A A): Option[A] A = {
case Some(data) modify(data)
case None modify(initial)
private def modifyWithInitial[A <: ReplicatedData](initial: A, modify: A => A): Option[A] => A = {
case Some(data) => modify(data)
case None => modify(initial)
}
}
/**
@ -529,7 +529,7 @@ object Replicator {
* for example not access `sender()` reference of an enclosing actor.
*/
final case class Update[A <: ReplicatedData](key: Key[A], writeConsistency: WriteConsistency,
request: Option[Any])(val modify: Option[A] A)
request: Option[Any])(val modify: Option[A] => A)
extends Command[A] with NoSerializationVerificationNeeded {
/**
@ -541,7 +541,7 @@ object Replicator {
*/
def this(
key: Key[A], initial: A, writeConsistency: WriteConsistency, modify: JFunction[A, A]) =
this(key, writeConsistency, None)(Update.modifyWithInitial(initial, data modify.apply(data)))
this(key, writeConsistency, None)(Update.modifyWithInitial(initial, data => modify.apply(data)))
/**
* Java API: Modify value of local `Replicator` and replicate with given `writeConsistency`.
@ -556,7 +556,7 @@ object Replicator {
*/
def this(
key: Key[A], initial: A, writeConsistency: WriteConsistency, request: Optional[Any], modify: JFunction[A, A]) =
this(key, writeConsistency, Option(request.orElse(null)))(Update.modifyWithInitial(initial, data modify.apply(data)))
this(key, writeConsistency, Option(request.orElse(null)))(Update.modifyWithInitial(initial, data => modify.apply(data)))
}
@ -720,8 +720,8 @@ object Replicator {
def needPruningFrom(removedNode: UniqueAddress): Boolean =
data match {
case r: RemovedNodePruning r.needPruningFrom(removedNode)
case _ false
case r: RemovedNodePruning => r.needPruningFrom(removedNode)
case _ => false
}
def initRemovedNodePruning(removed: UniqueAddress, owner: UniqueAddress): DataEnvelope = {
@ -732,18 +732,18 @@ object Replicator {
def prune(from: UniqueAddress, pruningPerformed: PruningPerformed): DataEnvelope = {
data match {
case dataWithRemovedNodePruning: RemovedNodePruning
case dataWithRemovedNodePruning: RemovedNodePruning =>
require(pruning.contains(from))
pruning(from) match {
case PruningInitialized(owner, _)
case PruningInitialized(owner, _) =>
val prunedData = dataWithRemovedNodePruning.prune(from, owner)
copy(data = prunedData, pruning = pruning.updated(from, pruningPerformed),
deltaVersions = cleanedDeltaVersions(from))
case _
case _ =>
this
}
case _ this
case _ => this
}
}
@ -752,11 +752,11 @@ object Replicator {
else {
val mergedPruning =
pruning.foldLeft(other.pruning) {
case (acc, (key, thisValue))
case (acc, (key, thisValue)) =>
acc.get(key) match {
case None
case None =>
acc.updated(key, thisValue)
case Some(thatValue)
case Some(thatValue) =>
acc.updated(key, thisValue merge thatValue)
}
}
@ -765,16 +765,16 @@ object Replicator {
else {
val currentTime = System.currentTimeMillis()
mergedPruning.filter {
case (_, p: PruningPerformed) !p.isObsolete(currentTime)
case _ true
case (_, p: PruningPerformed) => !p.isObsolete(currentTime)
case _ => true
}
}
}
// cleanup and merge deltaVersions
val removedNodes = filteredMergedPruning.keys
val cleanedDV = removedNodes.foldLeft(deltaVersions) { (acc, node) acc.pruningCleanup(node) }
val cleanedOtherDV = removedNodes.foldLeft(other.deltaVersions) { (acc, node) acc.pruningCleanup(node) }
val cleanedDV = removedNodes.foldLeft(deltaVersions) { (acc, node) => acc.pruningCleanup(node) }
val cleanedOtherDV = removedNodes.foldLeft(other.deltaVersions) { (acc, node) => acc.pruningCleanup(node) }
val mergedDeltaVersions = cleanedDV.merge(cleanedOtherDV)
// cleanup both sides before merging, `merge(otherData: ReplicatedData)` will cleanup other.data
@ -789,11 +789,11 @@ object Replicator {
else {
val mergedData =
cleaned(otherData, pruning) match {
case d: ReplicatedDelta data match {
case drd: DeltaReplicatedData drd.mergeDelta(d.asInstanceOf[drd.D])
case _ throw new IllegalArgumentException("Expected DeltaReplicatedData")
case d: ReplicatedDelta => data match {
case drd: DeltaReplicatedData => drd.mergeDelta(d.asInstanceOf[drd.D])
case _ => throw new IllegalArgumentException("Expected DeltaReplicatedData")
}
case c data.merge(c.asInstanceOf[data.T])
case c => data.merge(c.asInstanceOf[data.T])
}
if (data.getClass != mergedData.getClass)
throw new IllegalArgumentException(
@ -803,15 +803,15 @@ object Replicator {
}
private def cleaned(c: ReplicatedData, p: Map[UniqueAddress, PruningState]): ReplicatedData = p.foldLeft(c) {
case (c: RemovedNodePruning, (removed, _: PruningPerformed))
case (c: RemovedNodePruning, (removed, _: PruningPerformed)) =>
if (c.needPruningFrom(removed)) c.pruningCleanup(removed) else c
case (c, _) c
case (c, _) => c
}
def addSeen(node: Address): DataEnvelope = {
var changed = false
val newRemovedNodePruning = pruning.map {
case (removed, pruningState)
case (removed, pruningState) =>
val newPruningState = pruningState.addSeen(node)
changed = (newPruningState ne pruningState) || changed
(removed, newPruningState)
@ -831,7 +831,7 @@ object Replicator {
final case class Status(digests: Map[KeyId, Digest], chunk: Int, totChunks: Int) extends ReplicatorMessage {
override def toString: String =
(digests.map {
case (key, bytes) key + " -> " + bytes.map(byte f"$byte%02x").mkString("")
case (key, bytes) => key + " -> " + bytes.map(byte => f"$byte%02x").mkString("")
}).mkString("Status(", ", ", ")")
}
final case class Gossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean) extends ReplicatorMessage
@ -1085,12 +1085,12 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val hasDurableKeys = settings.durableKeys.nonEmpty
val durable = settings.durableKeys.filterNot(_.endsWith("*"))
val durableWildcards = settings.durableKeys.collect { case k if k.endsWith("*") k.dropRight(1) }
val durableWildcards = settings.durableKeys.collect { case k if k.endsWith("*") => k.dropRight(1) }
val durableStore: ActorRef =
if (hasDurableKeys) {
val props = settings.durableStoreProps match {
case Right(p) p
case Left((s, c))
case Right(p) => p
case Left((s, c)) =>
val clazz = context.system.asInstanceOf[ExtendedActorSystem].dynamicAccess.getClassFor[Actor](s).get
Props(clazz, c).withDispatcher(c.getString("use-dispatcher"))
}
@ -1111,10 +1111,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
// Important to include the pruning state in the deltas. For example if the delta is based
// on an entry that has been pruned but that has not yet been performed on the target node.
DeltaPropagation(selfUniqueAddress, reply = false, deltas.iterator.collect {
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder =>
getData(key) match {
case Some(envelope) key Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
case None key Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
case Some(envelope) => key -> Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
case None => key -> Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
}
}.toMap)
}
@ -1200,7 +1200,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def fromDurableStore: Boolean = sender() == durableStore && sender() != context.system.deadLetters
OneForOneStrategy()(
({
case e @ (_: DurableStore.LoadFailed | _: ActorInitializationException) if fromDurableStore
case e @ (_: DurableStore.LoadFailed | _: ActorInitializationException) if fromDurableStore =>
log.error(e, "Stopping distributed-data Replicator due to load or startup failure in durable store, caused by: {}", if (e.getCause eq null) "" else e.getCause.getMessage)
context.stop(self)
SupervisorStrategy.Stop
@ -1220,7 +1220,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def unstashAll(): Unit = {
val originalReplyTo = replyTo
stash.foreach {
case (msg, snd)
case (msg, snd) =>
replyTo = snd
normalReceive.applyOrElse(msg, unhandled)
}
@ -1229,18 +1229,18 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
{
case LoadData(data)
case LoadData(data) =>
count += data.size
data.foreach {
case (key, d)
case (key, d) =>
write(key, d.dataEnvelope) match {
case Some(newEnvelope)
case Some(newEnvelope) =>
if (newEnvelope ne d.dataEnvelope)
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
case None
case None =>
}
}
case LoadAllCompleted
case LoadAllCompleted =>
log.debug(
"Loading {} entries from durable store took {} ms, stashed {}",
count, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime), stash.size)
@ -1248,51 +1248,51 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
unstashAll()
self ! FlushChanges
case GetReplicaCount
case GetReplicaCount =>
// 0 until durable data has been loaded, used by test
replyTo ! ReplicaCount(0)
case RemovedNodePruningTick | FlushChanges | GossipTick
case RemovedNodePruningTick | FlushChanges | GossipTick =>
// ignore scheduled ticks when loading durable data
case TestFullStateGossip(enabled)
case TestFullStateGossip(enabled) =>
fullStateGossipEnabled = enabled
case m @ (_: Read | _: Write | _: Status | _: Gossip)
case m @ (_: Read | _: Write | _: Status | _: Gossip) =>
// ignore gossip and replication when loading durable data
log.debug("ignoring message [{}] when loading durable data", m.getClass.getName)
case msg: ClusterDomainEvent normalReceive.applyOrElse(msg, unhandled)
case msg
stash :+= (msg replyTo)
case msg: ClusterDomainEvent => normalReceive.applyOrElse(msg, unhandled)
case msg =>
stash :+= (msg -> replyTo)
}
}
// MUST use replyTo instead of sender() and forward from normalReceive, because of the stash in load
val normalReceive: Receive = {
case Get(key, consistency, req) receiveGet(key, consistency, req)
case u @ Update(key, writeC, req) receiveUpdate(key, u.modify, writeC, req)
case Read(key) receiveRead(key)
case Write(key, envelope) receiveWrite(key, envelope)
case ReadRepair(key, envelope) receiveReadRepair(key, envelope)
case DeltaPropagation(from, reply, deltas) receiveDeltaPropagation(from, reply, deltas)
case FlushChanges receiveFlushChanges()
case DeltaPropagationTick receiveDeltaPropagationTick()
case GossipTick receiveGossipTick()
case ClockTick receiveClockTick()
case Status(otherDigests, chunk, totChunks) receiveStatus(otherDigests, chunk, totChunks)
case Gossip(updatedData, sendBack) receiveGossip(updatedData, sendBack)
case Subscribe(key, subscriber) receiveSubscribe(key, subscriber)
case Unsubscribe(key, subscriber) receiveUnsubscribe(key, subscriber)
case Terminated(ref) receiveTerminated(ref)
case MemberWeaklyUp(m) receiveWeaklyUpMemberUp(m)
case MemberUp(m) receiveMemberUp(m)
case MemberRemoved(m, _) receiveMemberRemoved(m)
case evt: MemberEvent receiveOtherMemberEvent(evt.member)
case UnreachableMember(m) receiveUnreachable(m)
case ReachableMember(m) receiveReachable(m)
case GetKeyIds receiveGetKeyIds()
case Delete(key, consistency, req) receiveDelete(key, consistency, req)
case RemovedNodePruningTick receiveRemovedNodePruningTick()
case GetReplicaCount receiveGetReplicaCount()
case TestFullStateGossip(enabled) fullStateGossipEnabled = enabled
case Get(key, consistency, req) => receiveGet(key, consistency, req)
case u @ Update(key, writeC, req) => receiveUpdate(key, u.modify, writeC, req)
case Read(key) => receiveRead(key)
case Write(key, envelope) => receiveWrite(key, envelope)
case ReadRepair(key, envelope) => receiveReadRepair(key, envelope)
case DeltaPropagation(from, reply, deltas) => receiveDeltaPropagation(from, reply, deltas)
case FlushChanges => receiveFlushChanges()
case DeltaPropagationTick => receiveDeltaPropagationTick()
case GossipTick => receiveGossipTick()
case ClockTick => receiveClockTick()
case Status(otherDigests, chunk, totChunks) => receiveStatus(otherDigests, chunk, totChunks)
case Gossip(updatedData, sendBack) => receiveGossip(updatedData, sendBack)
case Subscribe(key, subscriber) => receiveSubscribe(key, subscriber)
case Unsubscribe(key, subscriber) => receiveUnsubscribe(key, subscriber)
case Terminated(ref) => receiveTerminated(ref)
case MemberWeaklyUp(m) => receiveWeaklyUpMemberUp(m)
case MemberUp(m) => receiveMemberUp(m)
case MemberRemoved(m, _) => receiveMemberRemoved(m)
case evt: MemberEvent => receiveOtherMemberEvent(evt.member)
case UnreachableMember(m) => receiveUnreachable(m)
case ReachableMember(m) => receiveReachable(m)
case GetKeyIds => receiveGetKeyIds()
case Delete(key, consistency, req) => receiveDelete(key, consistency, req)
case RemovedNodePruningTick => receiveRemovedNodePruningTick()
case GetReplicaCount => receiveGetReplicaCount()
case TestFullStateGossip(enabled) => fullStateGossipEnabled = enabled
}
def receiveGet(key: KeyR, consistency: ReadConsistency, req: Option[Any]): Unit = {
@ -1300,9 +1300,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
log.debug("Received Get for key [{}]", key)
if (isLocalGet(consistency)) {
val reply = localValue match {
case Some(DataEnvelope(DeletedData, _, _)) DataDeleted(key, req)
case Some(DataEnvelope(data, _, _)) GetSuccess(key, req)(data)
case None NotFound(key, req)
case Some(DataEnvelope(DeletedData, _, _)) => DataDeleted(key, req)
case Some(DataEnvelope(data, _, _)) => GetSuccess(key, req)(data)
case None => NotFound(key, req)
}
replyTo ! reply
} else
@ -1312,9 +1312,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def isLocalGet(readConsistency: ReadConsistency): Boolean =
readConsistency match {
case ReadLocal true
case _: ReadMajority | _: ReadAll nodes.isEmpty
case _ false
case ReadLocal => true
case _: ReadMajority | _: ReadAll => nodes.isEmpty
case _ => false
}
def receiveRead(key: KeyId): Unit = {
@ -1323,41 +1323,41 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def isLocalSender(): Boolean = !replyTo.path.address.hasGlobalScope
def receiveUpdate(key: KeyR, modify: Option[ReplicatedData] ReplicatedData,
def receiveUpdate(key: KeyR, modify: Option[ReplicatedData] => ReplicatedData,
writeConsistency: WriteConsistency, req: Option[Any]): Unit = {
val localValue = getData(key.id)
def deltaOrPlaceholder(d: DeltaReplicatedData): Option[ReplicatedDelta] = {
d.delta match {
case s @ Some(_) s
case None Some(NoDeltaPlaceholder)
case s @ Some(_) => s
case None => Some(NoDeltaPlaceholder)
}
}
Try {
localValue match {
case Some(DataEnvelope(DeletedData, _, _)) throw new DataDeleted(key, req)
case Some(envelope @ DataEnvelope(existing, _, _))
case Some(DataEnvelope(DeletedData, _, _)) => throw new DataDeleted(key, req)
case Some(envelope @ DataEnvelope(existing, _, _)) =>
modify(Some(existing)) match {
case d: DeltaReplicatedData if deltaCrdtEnabled
case d: DeltaReplicatedData if deltaCrdtEnabled =>
(envelope.merge(d.resetDelta.asInstanceOf[existing.T]), deltaOrPlaceholder(d))
case d
case d =>
(envelope.merge(d.asInstanceOf[existing.T]), None)
}
case None modify(None) match {
case d: DeltaReplicatedData if deltaCrdtEnabled
case None => modify(None) match {
case d: DeltaReplicatedData if deltaCrdtEnabled =>
(DataEnvelope(d.resetDelta), deltaOrPlaceholder(d))
case d (DataEnvelope(d), None)
case d => (DataEnvelope(d), None)
}
}
} match {
case Success((envelope, delta))
case Success((envelope, delta)) =>
log.debug("Received Update for key [{}]", key)
// handle the delta
delta match {
case Some(d) deltaPropagationSelector.update(key.id, d)
case None // not DeltaReplicatedData
case Some(d) => deltaPropagationSelector.update(key.id, d)
case None => // not DeltaReplicatedData
}
// note that it's important to do deltaPropagationSelector.update before setData,
@ -1373,12 +1373,12 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
replyTo ! UpdateSuccess(key, req)
} else {
val (writeEnvelope, writeDelta) = delta match {
case Some(NoDeltaPlaceholder) (newEnvelope, None)
case Some(d: RequiresCausalDeliveryOfDeltas)
case Some(NoDeltaPlaceholder) => (newEnvelope, None)
case Some(d: RequiresCausalDeliveryOfDeltas) =>
val v = deltaPropagationSelector.currentVersion(key.id)
(newEnvelope, Some(Delta(newEnvelope.copy(data = d), v, v)))
case Some(d) (newEnvelope.copy(data = d), None)
case None (newEnvelope, None)
case Some(d) => (newEnvelope.copy(data = d), None)
case None => (newEnvelope, None)
}
val writeAggregator =
context.actorOf(WriteAggregator.props(key, writeEnvelope, writeDelta, writeConsistency,
@ -1389,10 +1389,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), writeAggregator)))
}
}
case Failure(e: DataDeleted[_])
case Failure(e: DataDeleted[_]) =>
log.debug("Received Update for deleted key [{}]", key)
replyTo ! e
case Failure(e)
case Failure(e) =>
log.debug("Received Update for key [{}], failed: {}", key, e.getMessage)
replyTo ! ModifyFailure(key, "Update failed: " + e.getMessage, e, req)
}
@ -1403,9 +1403,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def isLocalUpdate(writeConsistency: WriteConsistency): Boolean =
writeConsistency match {
case WriteLocal true
case _: WriteMajority | _: WriteAll nodes.isEmpty
case _ false
case WriteLocal => true
case _: WriteMajority | _: WriteAll => nodes.isEmpty
case _ => false
}
def receiveWrite(key: KeyId, envelope: DataEnvelope): Unit =
@ -1413,13 +1413,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def writeAndStore(key: KeyId, writeEnvelope: DataEnvelope, reply: Boolean): Unit = {
write(key, writeEnvelope) match {
case Some(newEnvelope)
case Some(newEnvelope) =>
if (isDurable(key)) {
val storeReply = if (reply) Some(StoreReply(WriteAck, WriteNack, replyTo)) else None
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), storeReply)
} else if (reply)
replyTo ! WriteAck
case None
case None =>
if (reply)
replyTo ! WriteNack
}
@ -1427,27 +1427,27 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def write(key: KeyId, writeEnvelope: DataEnvelope): Option[DataEnvelope] = {
getData(key) match {
case someEnvelope @ Some(envelope) if envelope eq writeEnvelope someEnvelope
case Some(DataEnvelope(DeletedData, _, _)) Some(DeletedEnvelope) // already deleted
case Some(envelope @ DataEnvelope(existing, _, _))
case someEnvelope @ Some(envelope) if envelope eq writeEnvelope => someEnvelope
case Some(DataEnvelope(DeletedData, _, _)) => Some(DeletedEnvelope) // already deleted
case Some(envelope @ DataEnvelope(existing, _, _)) =>
try {
// DataEnvelope will mergeDelta when needed
val merged = envelope.merge(writeEnvelope).addSeen(selfAddress)
Some(setData(key, merged))
} catch {
case e: IllegalArgumentException
case e: IllegalArgumentException =>
log.warning(
"Couldn't merge [{}], due to: {}", key, e.getMessage)
None
}
case None
case None =>
// no existing data for the key
val writeEnvelope2 =
writeEnvelope.data match {
case d: ReplicatedDelta
case d: ReplicatedDelta =>
val z = d.zero
writeEnvelope.copy(data = z.mergeDelta(d.asInstanceOf[z.D]))
case _
case _ =>
writeEnvelope
}
@ -1463,17 +1463,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def receiveGetKeyIds(): Unit = {
val keys: Set[KeyId] = dataEntries.iterator.collect {
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData key
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData => key
}.to(immutable.Set)
replyTo ! GetKeyIdsResult(keys)
}
def receiveDelete(key: KeyR, consistency: WriteConsistency, req: Option[Any]): Unit = {
getData(key.id) match {
case Some(DataEnvelope(DeletedData, _, _))
case Some(DataEnvelope(DeletedData, _, _)) =>
// already deleted
replyTo ! DataDeleted(key, req)
case _
case _ =>
setData(key.id, DeletedEnvelope)
val durable = isDurable(key.id)
if (isLocalUpdate(consistency)) {
@ -1524,12 +1524,12 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def getDigest(key: KeyId): Digest = {
dataEntries.get(key) match {
case Some((envelope, LazyDigest))
case Some((envelope, LazyDigest)) =>
val d = digest(envelope)
dataEntries = dataEntries.updated(key, (envelope, d))
d
case Some((_, digest)) digest
case None NotFoundDigest
case Some((_, digest)) => digest
case None => NotFoundDigest
}
}
@ -1540,18 +1540,18 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
ByteString.fromArray(MessageDigest.getInstance("SHA-1").digest(bytes))
}
def getData(key: KeyId): Option[DataEnvelope] = dataEntries.get(key).map { case (envelope, _) envelope }
def getData(key: KeyId): Option[DataEnvelope] = dataEntries.get(key).map { case (envelope, _) => envelope }
def getDeltaSeqNr(key: KeyId, fromNode: UniqueAddress): Long =
dataEntries.get(key) match {
case Some((DataEnvelope(_, _, deltaVersions), _)) deltaVersions.versionAt(fromNode)
case None 0L
case Some((DataEnvelope(_, _, deltaVersions), _)) => deltaVersions.versionAt(fromNode)
case None => 0L
}
def isNodeRemoved(node: UniqueAddress, keys: Iterable[KeyId]): Boolean = {
removedNodes.contains(node) || (keys.exists(key dataEntries.get(key) match {
case Some((DataEnvelope(_, pruning, _), _)) pruning.contains(node)
case None false
removedNodes.contains(node) || (keys.exists(key => dataEntries.get(key) match {
case Some((DataEnvelope(_, pruning, _), _)) => pruning.contains(node)
case None => false
}))
}
@ -1559,22 +1559,22 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def notify(keyId: KeyId, subs: mutable.Set[ActorRef]): Unit = {
val key = subscriptionKeys(keyId)
getData(keyId) match {
case Some(envelope)
case Some(envelope) =>
val msg = if (envelope.data == DeletedData) Deleted(key) else Changed(key)(envelope.data)
subs.foreach { _ ! msg }
case None
case None =>
}
}
if (subscribers.nonEmpty) {
for (key changed; if subscribers.contains(key); subs subscribers.get(key))
for (key <- changed; if subscribers.contains(key); subs <- subscribers.get(key))
notify(key, subs)
}
// Changed event is sent to new subscribers even though the key has not changed,
// i.e. send current value
if (newSubscribers.nonEmpty) {
for ((key, subs) newSubscribers) {
for ((key, subs) <- newSubscribers) {
notify(key, subs)
subs.foreach { subscribers.addBinding(key, _) }
}
@ -1586,7 +1586,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def receiveDeltaPropagationTick(): Unit = {
deltaPropagationSelector.collectPropagations().foreach {
case (node, deltaPropagation)
case (node, deltaPropagation) =>
// TODO split it to several DeltaPropagation if too many entries
if (deltaPropagation.deltas.nonEmpty)
replica(node) ! deltaPropagation
@ -1602,7 +1602,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val isDebugEnabled = log.isDebugEnabled
if (isDebugEnabled)
log.debug("Received DeltaPropagation from [{}], containing [{}]", fromNode.address,
deltas.collect { case (key, Delta(_, fromSeqNr, toSeqNr)) s"$key $fromSeqNr-$toSeqNr" }.mkString(", "))
deltas.collect { case (key, Delta(_, fromSeqNr, toSeqNr)) => s"$key $fromSeqNr-$toSeqNr" }.mkString(", "))
if (isNodeRemoved(fromNode, deltas.keys)) {
// Late message from a removed node.
@ -1611,7 +1611,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
"Skipping DeltaPropagation from [{}] because that node has been removed", fromNode.address)
} else {
deltas.foreach {
case (key, Delta(envelope @ DataEnvelope(_: RequiresCausalDeliveryOfDeltas, _, _), fromSeqNr, toSeqNr))
case (key, Delta(envelope @ DataEnvelope(_: RequiresCausalDeliveryOfDeltas, _, _), fromSeqNr, toSeqNr)) =>
val currentSeqNr = getDeltaSeqNr(key, fromNode)
if (currentSeqNr >= toSeqNr) {
if (isDebugEnabled) log.debug(
@ -1630,13 +1630,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val newEnvelope = envelope.copy(deltaVersions = VersionVector(fromNode, toSeqNr))
writeAndStore(key, newEnvelope, reply)
}
case (key, Delta(envelope, _, _))
case (key, Delta(envelope, _, _)) =>
// causal delivery of deltas not needed, just apply it
writeAndStore(key, envelope, reply)
}
}
} catch {
case NonFatal(e)
case NonFatal(e) =>
// catching in case we need to support rolling upgrades that are
// mixing nodes with incompatible delta-CRDT types
log.warning("Couldn't process DeltaPropagation from [{}] due to {}", fromNode, e)
@ -1654,11 +1654,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def gossipTo(address: Address): Unit = {
val to = replica(address)
if (dataEntries.size <= maxDeltaElements) {
val status = Status(dataEntries.map { case (key, (_, _)) (key, getDigest(key)) }, chunk = 0, totChunks = 1)
val status = Status(dataEntries.map { case (key, (_, _)) => (key, getDigest(key)) }, chunk = 0, totChunks = 1)
to ! status
} else {
val totChunks = dataEntries.size / maxDeltaElements
for (_ 1 to math.min(totChunks, 10)) {
for (_ <- 1 to math.min(totChunks, 10)) {
if (totChunks == statusTotChunks)
statusCount += 1
else {
@ -1667,7 +1667,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
val chunk = (statusCount % totChunks).toInt
val status = Status(dataEntries.collect {
case (key, (_, _)) if math.abs(key.hashCode % totChunks) == chunk (key, getDigest(key))
case (key, (_, _)) if math.abs(key.hashCode % totChunks) == chunk => (key, getDigest(key))
}, chunk, totChunks)
to ! status
}
@ -1690,25 +1690,25 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
d != NotFoundDigest && d != otherDigest
}
val otherDifferentKeys = otherDigests.collect {
case (key, otherDigest) if isOtherDifferent(key, otherDigest) key
case (key, otherDigest) if isOtherDifferent(key, otherDigest) => key
}
val otherKeys = otherDigests.keySet
val myKeys =
if (totChunks == 1) dataEntries.keySet
else dataEntries.keysIterator.filter(key math.abs(key.hashCode % totChunks) == chunk).toSet
else dataEntries.keysIterator.filter(key => math.abs(key.hashCode % totChunks) == chunk).toSet
val otherMissingKeys = myKeys diff otherKeys
val keys = (otherDifferentKeys ++ otherMissingKeys).take(maxDeltaElements)
if (keys.nonEmpty) {
if (log.isDebugEnabled)
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
val g = Gossip(keys.iterator.map(k k getData(k).get).toMap, sendBack = otherDifferentKeys.nonEmpty)
val g = Gossip(keys.iterator.map(k => k -> getData(k).get).toMap, sendBack = otherDifferentKeys.nonEmpty)
replyTo ! g
}
val myMissingKeys = otherKeys diff myKeys
if (myMissingKeys.nonEmpty) {
if (log.isDebugEnabled)
log.debug("Sending gossip status to [{}], requesting missing [{}]", replyTo.path.address, myMissingKeys.mkString(", "))
val status = Status(myMissingKeys.iterator.map(k k NotFoundDigest).toMap, chunk, totChunks)
val status = Status(myMissingKeys.iterator.map(k => k -> NotFoundDigest).toMap, chunk, totChunks)
replyTo ! status
}
}
@ -1718,14 +1718,14 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
var replyData = Map.empty[KeyId, DataEnvelope]
updatedData.foreach {
case (key, envelope)
case (key, envelope) =>
val hadData = dataEntries.contains(key)
writeAndStore(key, envelope, reply = false)
if (sendBack) getData(key) match {
case Some(d)
case Some(d) =>
if (hadData || d.pruning.nonEmpty)
replyData = replyData.updated(key, d)
case None
case None =>
}
}
if (sendBack && replyData.nonEmpty)
@ -1749,20 +1749,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
def hasSubscriber(subscriber: ActorRef): Boolean =
(subscribers.exists { case (k, s) s.contains(subscriber) }) ||
(newSubscribers.exists { case (k, s) s.contains(subscriber) })
(subscribers.exists { case (k, s) => s.contains(subscriber) }) ||
(newSubscribers.exists { case (k, s) => s.contains(subscriber) })
def receiveTerminated(ref: ActorRef): Unit = {
if (ref == durableStore) {
log.error("Stopping distributed-data Replicator because durable store terminated")
context.stop(self)
} else {
val keys1 = subscribers.collect { case (k, s) if s.contains(ref) k }
keys1.foreach { key subscribers.removeBinding(key, ref) }
val keys2 = newSubscribers.collect { case (k, s) if s.contains(ref) k }
keys2.foreach { key newSubscribers.removeBinding(key, ref) }
val keys1 = subscribers.collect { case (k, s) if s.contains(ref) => k }
keys1.foreach { key => subscribers.removeBinding(key, ref) }
val keys2 = newSubscribers.collect { case (k, s) if s.contains(ref) => k }
keys2.foreach { key => newSubscribers.removeBinding(key, ref) }
(keys1 ++ keys2).foreach { key
(keys1 ++ keys2).foreach { key =>
if (!subscribers.contains(key) && !newSubscribers.contains(key))
subscriptionKeys -= key
}
@ -1833,13 +1833,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val knownNodes = nodes union weaklyUpNodes union removedNodes.keySet.map(_.address)
val newRemovedNodes =
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
case (acc, (_, (envelope @ DataEnvelope(data: RemovedNodePruning, _, _), _)))
acc union data.modifiedByNodes.filterNot(n n == selfUniqueAddress || knownNodes(n.address))
case (acc, _)
case (acc, (_, (envelope @ DataEnvelope(data: RemovedNodePruning, _, _), _))) =>
acc union data.modifiedByNodes.filterNot(n => n == selfUniqueAddress || knownNodes(n.address))
case (acc, _) =>
acc
}
newRemovedNodes.foreach { n
newRemovedNodes.foreach { n =>
log.debug("Adding removed node [{}] from data", n)
removedNodes = removedNodes.updated(n, allReachableClockTime)
}
@ -1848,11 +1848,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def initRemovedNodePruning(): Unit = {
// initiate pruning for removed nodes
val removedSet: Set[UniqueAddress] = removedNodes.iterator.collect {
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) r
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) => r
}.to(immutable.Set)
if (removedSet.nonEmpty) {
for ((key, (envelope, _)) dataEntries; removed removedSet) {
for ((key, (envelope, _)) <- dataEntries; removed <- removedSet) {
def init(): Unit = {
val newEnvelope = envelope.initRemovedNodePruning(removed, selfUniqueAddress)
@ -1862,13 +1862,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
if (envelope.needPruningFrom(removed)) {
envelope.data match {
case dataWithRemovedNodePruning: RemovedNodePruning
case dataWithRemovedNodePruning: RemovedNodePruning =>
envelope.pruning.get(removed) match {
case None init()
case Some(PruningInitialized(owner, _)) if owner != selfUniqueAddress init()
case _ // already in progress
case None => init()
case Some(PruningInitialized(owner, _)) if owner != selfUniqueAddress => init()
case _ => // already in progress
}
case _
case _ =>
}
}
}
@ -1881,36 +1881,36 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
dataEntries.foreach {
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, pruning, _), _))
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, pruning, _), _)) =>
pruning.foreach {
case (removed, PruningInitialized(owner, seen)) if owner == selfUniqueAddress
&& (allNodes.isEmpty || allNodes.forall(seen))
&& (allNodes.isEmpty || allNodes.forall(seen)) =>
val newEnvelope = envelope.prune(removed, if (isDurable(key)) durablePruningPerformed else pruningPerformed)
log.debug("Perform pruning of [{}] from [{}] to [{}]", key, removed, selfUniqueAddress)
setData(key, newEnvelope)
if ((newEnvelope.data ne data) && isDurable(key))
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
case _
case _ =>
}
case _ // deleted, or pruning not needed
case _ => // deleted, or pruning not needed
}
}
def deleteObsoletePruningPerformed(): Unit = {
val currentTime = System.currentTimeMillis()
dataEntries.foreach {
case (key, (envelope @ DataEnvelope(_: RemovedNodePruning, pruning, _), _))
case (key, (envelope @ DataEnvelope(_: RemovedNodePruning, pruning, _), _)) =>
val newEnvelope = pruning.foldLeft(envelope) {
case (acc, (removed, p: PruningPerformed)) if p.isObsolete(currentTime)
case (acc, (removed, p: PruningPerformed)) if p.isObsolete(currentTime) =>
log.debug("Removing obsolete pruning marker for [{}] in [{}]", removed, key)
removedNodes -= removed
acc.copy(pruning = acc.pruning - removed)
case (acc, _) acc
case (acc, _) => acc
}
if (newEnvelope ne envelope)
setData(key, newEnvelope)
case _ // deleted, or pruning not needed
case _ => // deleted, or pruning not needed
}
}
@ -2026,20 +2026,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
override def timeout: FiniteDuration = consistency.timeout
override val doneWhenRemainingSize = consistency match {
case WriteTo(n, _) nodes.size - (n - 1)
case _: WriteAll 0
case WriteMajority(_, minCap)
case WriteTo(n, _) => nodes.size - (n - 1)
case _: WriteAll => 0
case WriteMajority(_, minCap) =>
val N = nodes.size + 1
val w = calculateMajorityWithMinCap(minCap, N)
N - w
case WriteLocal
case WriteLocal =>
throw new IllegalArgumentException("WriteLocal not supported by WriteAggregator")
}
val writeMsg = Write(key.id, envelope)
val deltaMsg = delta match {
case None None
case Some(d) Some(DeltaPropagation(selfUniqueAddress, reply = true, Map(key.id d)))
case None => None
case Some(d) => Some(DeltaPropagation(selfUniqueAddress, reply = true, Map(key.id -> d)))
}
var gotLocalStoreReply = !durable
@ -2047,16 +2047,16 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
private val (primaryNodes, secondaryNodes) = {
val requiresCausalDeliveryOfDeltas = delta match {
case None false
case Some(d) d.dataEnvelope.data.isInstanceOf[RequiresCausalDeliveryOfDeltas]
case None => false
case Some(d) => d.dataEnvelope.data.isInstanceOf[RequiresCausalDeliveryOfDeltas]
}
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas)
}
override def preStart(): Unit = {
val msg = deltaMsg match {
case Some(d) d
case None writeMsg
case Some(d) => d
case None => writeMsg
}
primaryNodes.foreach { replica(_) ! msg }
@ -2064,37 +2064,37 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
def receive: Receive = {
case WriteAck
case WriteAck =>
remaining -= senderAddress()
if (isDone) reply(isTimeout = false)
case WriteNack
case WriteNack =>
gotWriteNackFrom += senderAddress()
if (isDone) reply(isTimeout = false)
case DeltaNack
case DeltaNack =>
// Deltas must be applied in order and we can't keep track of ordering of
// simultaneous updates so there is a chance that the delta could not be applied.
// Try again with the full state
sender() ! writeMsg
case _: Replicator.UpdateSuccess[_]
case _: Replicator.UpdateSuccess[_] =>
gotLocalStoreReply = true
if (isDone) reply(isTimeout = false)
case f: Replicator.StoreFailure[_]
case f: Replicator.StoreFailure[_] =>
gotLocalStoreReply = true
gotWriteNackFrom += selfUniqueAddress.address
if (isDone) reply(isTimeout = false)
case SendToSecondary
case SendToSecondary =>
deltaMsg match {
case None
case Some(d)
case None =>
case Some(d) =>
// Deltas must be applied in order and we can't keep track of ordering of
// simultaneous updates so there is a chance that the delta could not be applied.
// Try again with the full state to the primary nodes that have not acked.
primaryNodes.toSet.intersect(remaining).foreach { replica(_) ! writeMsg }
}
secondaryNodes.foreach { replica(_) ! writeMsg }
case ReceiveTimeout
case ReceiveTimeout =>
reply(isTimeout = true)
}
@ -2162,13 +2162,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
var result = localValue
override val doneWhenRemainingSize = consistency match {
case ReadFrom(n, _) nodes.size - (n - 1)
case _: ReadAll 0
case ReadMajority(_, minCap)
case ReadFrom(n, _) => nodes.size - (n - 1)
case _: ReadAll => 0
case ReadMajority(_, minCap) =>
val N = nodes.size + 1
val r = calculateMajorityWithMinCap(minCap, N)
N - r
case ReadLocal
case ReadLocal =>
throw new IllegalArgumentException("ReadLocal not supported by ReadAggregator")
}
@ -2188,47 +2188,47 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
def receive = {
case ReadResult(envelope)
case ReadResult(envelope) =>
result = (result, envelope) match {
case (Some(a), Some(b)) Some(a.merge(b))
case (r @ Some(_), None) r
case (None, r @ Some(_)) r
case (None, None) None
case (Some(a), Some(b)) => Some(a.merge(b))
case (r @ Some(_), None) => r
case (None, r @ Some(_)) => r
case (None, None) => None
}
remaining -= sender().path.address
if (remaining.size == doneWhenRemainingSize)
reply(ok = true)
case SendToSecondary
case SendToSecondary =>
secondaryNodes.foreach { replica(_) ! readMsg }
case ReceiveTimeout reply(ok = false)
case ReceiveTimeout => reply(ok = false)
}
def reply(ok: Boolean): Unit =
(ok, result) match {
case (true, Some(envelope))
case (true, Some(envelope)) =>
context.parent ! ReadRepair(key.id, envelope)
// read-repair happens before GetSuccess
context.become(waitReadRepairAck(envelope))
case (true, None)
case (true, None) =>
replyTo.tell(NotFound(key, req), context.parent)
context.stop(self)
case (false, _)
case (false, _) =>
replyTo.tell(GetFailure(key, req), context.parent)
context.stop(self)
}
def waitReadRepairAck(envelope: Replicator.Internal.DataEnvelope): Receive = {
case ReadRepairAck
case ReadRepairAck =>
val replyMsg =
if (envelope.data == DeletedData) DataDeleted(key, req)
else GetSuccess(key, req)(envelope.data)
replyTo.tell(replyMsg, context.parent)
context.stop(self)
case _: ReadResult
case _: ReadResult =>
//collect late replies
remaining -= sender().path.address
case SendToSecondary
case ReceiveTimeout
case SendToSecondary =>
case ReceiveTimeout =>
}
}