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:
parent
c5d18c30d6
commit
952be31a7d
28 changed files with 951 additions and 229 deletions
|
|
@ -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)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue