make pruning of CRDT garbage work, #21647

* fix merge issues of DataEnvelope and its pruning
* simplify by removing the tombstones, which didn't work in all cases anyway
* keep the PruningPerformed markers in the DataEnvelope until configured
  TTL has elapsed (wall clock)
* simplify PruningState structure
* also store the pruning markers in durable data
* collect removed nodes from the data, listing on MemberRemoved is not enough
* possibility to disable pruning altogether
* documented caveat for durable data
This commit is contained in:
Patrik Nordwall 2017-01-11 13:19:45 +01:00
parent c5d18c30d6
commit 952be31a7d
28 changed files with 951 additions and 229 deletions

View file

@ -43,6 +43,7 @@ import akka.actor.SupervisorStrategy
import akka.actor.OneForOneStrategy
import akka.actor.ActorInitializationException
import java.util.concurrent.TimeUnit
import akka.util.Helpers.toRootLowerCase
object ReplicatorSettings {
@ -63,6 +64,11 @@ object ReplicatorSettings {
case id id
}
val pruningInterval = toRootLowerCase(config.getString("pruning-interval")) match {
case "off" | "false" Duration.Zero
case _ config.getDuration("pruning-interval", MILLISECONDS).millis
}
import scala.collection.JavaConverters._
new ReplicatorSettings(
role = roleOption(config.getString("role")),
@ -70,10 +76,12 @@ object ReplicatorSettings {
notifySubscribersInterval = config.getDuration("notify-subscribers-interval", MILLISECONDS).millis,
maxDeltaElements = config.getInt("max-delta-elements"),
dispatcher = dispatcher,
pruningInterval = config.getDuration("pruning-interval", MILLISECONDS).millis,
pruningInterval = pruningInterval,
maxPruningDissemination = config.getDuration("max-pruning-dissemination", MILLISECONDS).millis,
durableStoreProps = Left((config.getString("durable.store-actor-class"), config.getConfig("durable"))),
durableKeys = config.getStringList("durable.keys").asScala.toSet)
durableKeys = config.getStringList("durable.keys").asScala.toSet,
pruningMarkerTimeToLive = config.getDuration("pruning-marker-time-to-live", MILLISECONDS).millis,
durablePruningMarkerTimeToLive = config.getDuration("durable.pruning-marker-time-to-live", MILLISECONDS).millis)
}
/**
@ -110,21 +118,30 @@ object ReplicatorSettings {
* in the `Set`.
*/
final class ReplicatorSettings(
val role: Option[String],
val gossipInterval: FiniteDuration,
val notifySubscribersInterval: FiniteDuration,
val maxDeltaElements: Int,
val dispatcher: String,
val pruningInterval: FiniteDuration,
val maxPruningDissemination: FiniteDuration,
val durableStoreProps: Either[(String, Config), Props],
val durableKeys: Set[String]) {
val role: Option[String],
val gossipInterval: FiniteDuration,
val notifySubscribersInterval: FiniteDuration,
val maxDeltaElements: Int,
val dispatcher: String,
val pruningInterval: FiniteDuration,
val maxPruningDissemination: FiniteDuration,
val durableStoreProps: Either[(String, Config), Props],
val durableKeys: Set[String],
val pruningMarkerTimeToLive: FiniteDuration,
val durablePruningMarkerTimeToLive: FiniteDuration) {
// For backwards compatibility
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration) =
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
maxPruningDissemination, Right(Props.empty), Set.empty)
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days)
// For backwards compatibility
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration,
durableStoreProps: Either[(String, Config), Props], durableKeys: Set[String]) =
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
maxPruningDissemination, durableStoreProps, durableKeys, 6.hours, 10.days)
def withRole(role: String): ReplicatorSettings = copy(role = ReplicatorSettings.roleOption(role))
@ -150,6 +167,13 @@ final class ReplicatorSettings(
def withPruning(pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration): ReplicatorSettings =
copy(pruningInterval = pruningInterval, maxPruningDissemination = maxPruningDissemination)
def withPruningMarkerTimeToLive(
pruningMarkerTimeToLive: FiniteDuration,
durablePruningMarkerTimeToLive: FiniteDuration): ReplicatorSettings =
copy(
pruningMarkerTimeToLive = pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive = durablePruningMarkerTimeToLive)
def withDurableStoreProps(durableStoreProps: Props): ReplicatorSettings =
copy(durableStoreProps = Right(durableStoreProps))
@ -168,17 +192,20 @@ final class ReplicatorSettings(
}
private def copy(
role: Option[String] = role,
gossipInterval: FiniteDuration = gossipInterval,
notifySubscribersInterval: FiniteDuration = notifySubscribersInterval,
maxDeltaElements: Int = maxDeltaElements,
dispatcher: String = dispatcher,
pruningInterval: FiniteDuration = pruningInterval,
maxPruningDissemination: FiniteDuration = maxPruningDissemination,
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
durableKeys: Set[String] = durableKeys): ReplicatorSettings =
role: Option[String] = role,
gossipInterval: FiniteDuration = gossipInterval,
notifySubscribersInterval: FiniteDuration = notifySubscribersInterval,
maxDeltaElements: Int = maxDeltaElements,
dispatcher: String = dispatcher,
pruningInterval: FiniteDuration = pruningInterval,
maxPruningDissemination: FiniteDuration = maxPruningDissemination,
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
durableKeys: Set[String] = durableKeys,
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive): ReplicatorSettings =
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys)
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys,
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive)
}
object Replicator {
@ -562,6 +589,9 @@ object Replicator {
val LazyDigest: Digest = ByteString(0)
val NotFoundDigest: Digest = ByteString(-1)
/**
* The `DataEnvelope` wraps a data entry and carries state of the pruning process for the entry.
*/
final case class DataEnvelope(
data: ReplicatedData,
pruning: Map[UniqueAddress, PruningState] = Map.empty)
@ -576,36 +606,51 @@ object Replicator {
}
def initRemovedNodePruning(removed: UniqueAddress, owner: UniqueAddress): DataEnvelope = {
copy(pruning = pruning.updated(removed, PruningState(owner, PruningInitialized(Set.empty))))
copy(pruning = pruning.updated(removed, PruningInitialized(owner, Set.empty)))
}
def prune(from: UniqueAddress): DataEnvelope = {
def prune(from: UniqueAddress, pruningPerformed: PruningPerformed): DataEnvelope = {
data match {
case dataWithRemovedNodePruning: RemovedNodePruning
require(pruning.contains(from))
val to = pruning(from).owner
val prunedData = dataWithRemovedNodePruning.prune(from, to)
copy(data = prunedData, pruning = pruning.updated(from, PruningState(to, PruningPerformed)))
pruning(from) match {
case PruningInitialized(owner, _)
val prunedData = dataWithRemovedNodePruning.prune(from, owner)
copy(data = prunedData, pruning = pruning.updated(from, pruningPerformed))
case _
this
}
case _ this
}
}
def merge(other: DataEnvelope): DataEnvelope =
if (other.data == DeletedData) DeletedEnvelope
else {
var mergedRemovedNodePruning = other.pruning
for ((key, thisValue) pruning) {
mergedRemovedNodePruning.get(key) match {
case None
mergedRemovedNodePruning = mergedRemovedNodePruning.updated(key, thisValue)
case Some(thatValue)
mergedRemovedNodePruning = mergedRemovedNodePruning.updated(key, thisValue merge thatValue)
val mergedPruning =
pruning.foldLeft(other.pruning) {
case (acc, (key, thisValue))
acc.get(key) match {
case None
acc.updated(key, thisValue)
case Some(thatValue)
acc.updated(key, thisValue merge thatValue)
}
}
val filteredMergedPruning = {
if (mergedPruning.isEmpty) mergedPruning
else {
val currentTime = System.currentTimeMillis()
mergedPruning.filter {
case (_, p: PruningPerformed) !p.isObsolete(currentTime)
case _ true
}
}
}
// cleanup both sides before merging, `merge((otherData: ReplicatedData)` will cleanup other.data
copy(data = cleaned(data, mergedRemovedNodePruning), pruning = mergedRemovedNodePruning).merge(other.data)
copy(data = cleaned(data, filteredMergedPruning), pruning = filteredMergedPruning).merge(other.data)
}
def merge(otherData: ReplicatedData): DataEnvelope =
@ -613,7 +658,7 @@ object Replicator {
else copy(data = data merge cleaned(otherData, pruning).asInstanceOf[data.T])
private def cleaned(c: ReplicatedData, p: Map[UniqueAddress, PruningState]): ReplicatedData = p.foldLeft(c) {
case (c: RemovedNodePruning, (removed, PruningState(_, PruningPerformed)))
case (c: RemovedNodePruning, (removed, _: PruningPerformed))
if (c.needPruningFrom(removed)) c.pruningCleanup(removed) else c
case (c, _) c
}
@ -801,7 +846,8 @@ object Replicator {
* <li>When a node is removed from the cluster it is first important that all updates that were
* done by that node are disseminated to all other nodes. The pruning will not start before the
* `maxPruningDissemination` duration has elapsed. The time measurement is stopped when any
* replica is unreachable, so it should be configured to worst case in a healthy cluster.</li>
* replica is unreachable, but it's still recommended to configure this with certain margin.
* It should be in the magnitude of minutes.</li>
* <li>The nodes are ordered by their address and the node ordered first is called leader.
* The leader initiates the pruning by adding a `PruningInitialized` marker in the data envelope.
* This is gossiped to all other nodes and they mark it as seen when they receive it.</li>
@ -840,7 +886,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
import context.dispatcher
val gossipTask = context.system.scheduler.schedule(gossipInterval, gossipInterval, self, GossipTick)
val notifyTask = context.system.scheduler.schedule(notifySubscribersInterval, notifySubscribersInterval, self, FlushChanges)
val pruningTask = context.system.scheduler.schedule(pruningInterval, pruningInterval, self, RemovedNodePruningTick)
val pruningTask =
if (pruningInterval >= Duration.Zero)
Some(context.system.scheduler.schedule(pruningInterval, pruningInterval, self, RemovedNodePruningTick))
else None
val clockTask = context.system.scheduler.schedule(gossipInterval, gossipInterval, self, ClockTick)
val serializer = SerializationExtension(context.system).serializerFor(classOf[DataEnvelope])
@ -867,11 +916,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
// cluster weaklyUp nodes, doesn't contain selfAddress
var weaklyUpNodes: Set[Address] = Set.empty
// nodes removed from cluster, to be pruned, and tombstoned
var removedNodes: Map[UniqueAddress, Long] = Map.empty
var pruningPerformed: Map[UniqueAddress, Long] = Map.empty
var tombstoneNodes: Set[UniqueAddress] = Set.empty
var leader: Option[Address] = None
def isLeader: Boolean = leader.exists(_ == selfAddress)
@ -921,7 +966,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
cluster.unsubscribe(self)
gossipTask.cancel()
notifyTask.cancel()
pruningTask.cancel()
pruningTask.foreach(_.cancel())
clockTask.cancel()
}
@ -964,11 +1009,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
count += data.size
data.foreach {
case (key, d)
val envelope = DataEnvelope(d)
write(key, envelope) match {
write(key, d.dataEnvelope) match {
case Some(newEnvelope)
if (newEnvelope.data ne envelope.data)
durableStore ! Store(key, newEnvelope.data, None)
if (newEnvelope.data ne d.dataEnvelope.data)
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
case None
}
}
@ -1059,18 +1103,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
localValue match {
case Some(DataEnvelope(DeletedData, _)) throw new DataDeleted(key, req)
case Some(envelope @ DataEnvelope(existing, _))
existing.merge(modify(Some(existing)).asInstanceOf[existing.T])
case None modify(None)
envelope.merge(modify(Some(existing)).asInstanceOf[existing.T])
case None DataEnvelope(modify(None))
}
} match {
case Success(newData)
log.debug("Received Update for key [{}], old data [{}], new data [{}]", key, localValue, newData)
val envelope = DataEnvelope(pruningCleanupTombstoned(newData))
case Success(envelope)
log.debug("Received Update for key [{}], old data [{}], new data [{}]", key, localValue, envelope.data)
setData(key.id, envelope)
val durable = isDurable(key.id)
if (isLocalUpdate(writeConsistency)) {
if (durable)
durableStore ! Store(key.id, envelope.data,
durableStore ! Store(key.id, new DurableDataEnvelope(envelope),
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), replyTo)))
else
replyTo ! UpdateSuccess(key, req)
@ -1079,7 +1122,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
context.actorOf(WriteAggregator.props(key, envelope, writeConsistency, req, nodes, unreachable, replyTo, durable)
.withDispatcher(context.props.dispatcher))
if (durable) {
durableStore ! Store(key.id, envelope.data,
durableStore ! Store(key.id, new DurableDataEnvelope(envelope),
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), writeAggregator)))
}
}
@ -1106,7 +1149,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
write(key, envelope) match {
case Some(newEnvelope)
if (isDurable(key))
durableStore ! Store(key, newEnvelope.data, Some(StoreReply(WriteAck, WriteNack, replyTo)))
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), Some(StoreReply(WriteAck, WriteNack, replyTo)))
else
replyTo ! WriteAck
case None
@ -1115,10 +1158,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def write(key: String, writeEnvelope: DataEnvelope): Option[DataEnvelope] =
getData(key) match {
case Some(DataEnvelope(DeletedData, _)) Some(writeEnvelope) // already deleted
case Some(DataEnvelope(DeletedData, _)) Some(DeletedEnvelope) // already deleted
case Some(envelope @ DataEnvelope(existing, _))
if (existing.getClass == writeEnvelope.data.getClass || writeEnvelope.data == DeletedData) {
val merged = envelope.merge(pruningCleanupTombstoned(writeEnvelope)).addSeen(selfAddress)
val merged = envelope.merge(writeEnvelope).addSeen(selfAddress)
setData(key, merged)
Some(merged)
} else {
@ -1128,16 +1171,16 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
None
}
case None
val cleaned = pruningCleanupTombstoned(writeEnvelope).addSeen(selfAddress)
setData(key, cleaned)
Some(cleaned)
val writeEnvelope2 = writeEnvelope.addSeen(selfAddress)
setData(key, writeEnvelope2)
Some(writeEnvelope2)
}
def receiveReadRepair(key: String, writeEnvelope: DataEnvelope): Unit = {
write(key, writeEnvelope) match {
case Some(newEnvelope)
if (isDurable(key))
durableStore ! Store(key, newEnvelope.data, None)
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
case None
}
replyTo ! ReadRepairAck
@ -1160,7 +1203,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val durable = isDurable(key.id)
if (isLocalUpdate(consistency)) {
if (durable)
durableStore ! Store(key.id, DeletedData,
durableStore ! Store(key.id, new DurableDataEnvelope(DeletedEnvelope),
Some(StoreReply(DeleteSuccess(key, req), StoreFailure(key, req), replyTo)))
else
replyTo ! DeleteSuccess(key, req)
@ -1169,7 +1212,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, req, nodes, unreachable, replyTo, durable)
.withDispatcher(context.props.dispatcher))
if (durable) {
durableStore ! Store(key.id, DeletedData,
durableStore ! Store(key.id, new DurableDataEnvelope(DeletedEnvelope),
Some(StoreReply(DeleteSuccess(key, req), StoreFailure(key, req), writeAggregator)))
}
}
@ -1313,7 +1356,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
write(key, envelope) match {
case Some(newEnvelope)
if (isDurable(key))
durableStore ! Store(key, newEnvelope.data, None)
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
case None
}
if (sendBack) getData(key) match {
@ -1380,6 +1423,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
else if (matchingRole(m)) {
nodes -= m.address
weaklyUpNodes -= m.address
log.debug("adding removed node [{}] from MemberRemoved", m.uniqueAddress)
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
unreachable -= m.address
}
@ -1402,12 +1446,31 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
def receiveRemovedNodePruningTick(): Unit = {
if (isLeader && removedNodes.nonEmpty) {
initRemovedNodePruning()
// See 'CRDT Garbage' section in Replicator Scaladoc for description of the process
if (unreachable.isEmpty) {
if (isLeader) {
collectRemovedNodes()
initRemovedNodePruning()
}
performRemovedNodePruning()
deleteObsoletePruningPerformed()
}
}
def collectRemovedNodes(): Unit = {
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, _)
acc
}
newRemovedNodes.foreach { n
log.debug("Adding removed node [{}] from data", n)
removedNodes = removedNodes.updated(n, allReachableClockTime)
}
performRemovedNodePruning()
// FIXME tombstoneRemovedNodePruning doesn't work, since merge of PruningState will add the PruningPerformed back again
// tombstoneRemovedNodePruning()
}
def initRemovedNodePruning(): Unit = {
@ -1417,22 +1480,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}(collection.breakOut)
if (removedSet.nonEmpty) {
// FIXME handle pruning of durable data, this is difficult and requires more thought
for ((key, (envelope, _)) dataEntries; removed removedSet) {
def init(): Unit = {
val newEnvelope = envelope.initRemovedNodePruning(removed, selfUniqueAddress)
log.debug("Initiated pruning of [{}] for data key [{}]", removed, key)
log.debug("Initiated pruning of [{}] for data key [{}] to [{}]", removed, key, selfUniqueAddress)
setData(key, newEnvelope)
}
if (envelope.needPruningFrom(removed)) {
envelope.data match {
case dataWithRemovedNodePruning: RemovedNodePruning
envelope.pruning.get(removed) match {
case None init()
case Some(PruningState(owner, PruningInitialized(_))) if owner != selfUniqueAddress init()
case Some(PruningInitialized(owner, _)) if owner != selfUniqueAddress init()
case _ // already in progress
}
case _
@ -1444,78 +1505,44 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def performRemovedNodePruning(): Unit = {
// perform pruning when all seen Init
val allNodes = nodes union weaklyUpNodes
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
dataEntries.foreach {
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, pruning), _))
pruning.foreach {
case (removed, PruningState(owner, PruningInitialized(seen))) if owner == selfUniqueAddress
&& (nodes.isEmpty || nodes.forall(seen))
val newEnvelope = envelope.prune(removed)
pruningPerformed = pruningPerformed.updated(removed, allReachableClockTime)
case (removed, PruningInitialized(owner, seen)) if owner == selfUniqueAddress
&& (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, newEnvelope.data, None)
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
case _
}
case _ // deleted, or pruning not needed
}
}
def tombstoneRemovedNodePruning(): Unit = {
def allPruningPerformed(removed: UniqueAddress): Boolean = {
dataEntries forall {
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, pruning), _))
pruning.get(removed) match {
case Some(PruningState(_, PruningInitialized(_))) false
case _ true
}
case _ true // deleted, or pruning not needed
}
}
// FIXME pruningPerformed is only updated on one node, but tombstoneNodes should be on all
pruningPerformed.foreach {
case (removed, timestamp) if ((allReachableClockTime - timestamp) > maxPruningDisseminationNanos) &&
allPruningPerformed(removed)
log.debug("All pruning performed for [{}], tombstoned", removed)
pruningPerformed -= removed
removedNodes -= removed
tombstoneNodes += removed
dataEntries.foreach {
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, _), _))
val newEnvelope = pruningCleanupTombstoned(removed, envelope)
setData(key, newEnvelope)
if ((newEnvelope.data ne data) && isDurable(key))
durableStore ! Store(key, newEnvelope.data, None)
case _ // deleted, or pruning not needed
def deleteObsoletePruningPerformed(): Unit = {
val currentTime = System.currentTimeMillis()
dataEntries.foreach {
case (key, (envelope @ DataEnvelope(_: RemovedNodePruning, pruning), _))
val newEnvelope = pruning.foldLeft(envelope) {
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 (removed, timestamp) // not ready
if (newEnvelope ne envelope)
setData(key, newEnvelope)
case _ // deleted, or pruning not needed
}
}
def pruningCleanupTombstoned(envelope: DataEnvelope): DataEnvelope =
tombstoneNodes.foldLeft(envelope)((c, removed) pruningCleanupTombstoned(removed, c))
def pruningCleanupTombstoned(removed: UniqueAddress, envelope: DataEnvelope): DataEnvelope = {
val pruningCleanuped = pruningCleanupTombstoned(removed, envelope.data)
if ((pruningCleanuped ne envelope.data) || envelope.pruning.contains(removed))
envelope.copy(data = pruningCleanuped, pruning = envelope.pruning - removed)
else
envelope
}
def pruningCleanupTombstoned(data: ReplicatedData): ReplicatedData =
if (tombstoneNodes.isEmpty) data
else tombstoneNodes.foldLeft(data)((c, removed) pruningCleanupTombstoned(removed, c))
def pruningCleanupTombstoned(removed: UniqueAddress, data: ReplicatedData): ReplicatedData =
data match {
case dataWithRemovedNodePruning: RemovedNodePruning
if (dataWithRemovedNodePruning.needPruningFrom(removed)) dataWithRemovedNodePruning.pruningCleanup(removed) else data
case _ data
}
def receiveGetReplicaCount(): Unit = {
// selfAddress is not included in the set
replyTo ! ReplicaCount(nodes.size + 1)