Add support for durable storage of distributed data, #21645
* using lmdbjava libarary
This commit is contained in:
parent
446c0545ec
commit
d6d50a08d0
18 changed files with 1892 additions and 124 deletions
|
|
@ -4,7 +4,9 @@
|
|||
package akka.cluster.ddata
|
||||
|
||||
import java.security.MessageDigest
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.Queue
|
||||
import scala.collection.mutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
|
@ -37,6 +39,11 @@ import akka.dispatch.Dispatchers
|
|||
import akka.actor.DeadLetterSuppression
|
||||
import akka.cluster.ddata.Key.KeyR
|
||||
import java.util.Optional
|
||||
import akka.cluster.ddata.DurableStore._
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.actor.ActorInitializationException
|
||||
|
||||
object ReplicatorSettings {
|
||||
|
||||
|
|
@ -56,6 +63,8 @@ object ReplicatorSettings {
|
|||
case "" ⇒ Dispatchers.DefaultDispatcherId
|
||||
case id ⇒ id
|
||||
}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
new ReplicatorSettings(
|
||||
role = roleOption(config.getString("role")),
|
||||
gossipInterval = config.getDuration("gossip-interval", MILLISECONDS).millis,
|
||||
|
|
@ -63,7 +72,9 @@ object ReplicatorSettings {
|
|||
maxDeltaElements = config.getInt("max-delta-elements"),
|
||||
dispatcher = dispatcher,
|
||||
pruningInterval = config.getDuration("pruning-interval", MILLISECONDS).millis,
|
||||
maxPruningDissemination = config.getDuration("max-pruning-dissemination", MILLISECONDS).millis)
|
||||
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)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -91,6 +102,13 @@ object ReplicatorSettings {
|
|||
* completing the pruning process of data associated with removed cluster nodes.
|
||||
* The time measurement is stopped when any replica is unreachable, so it should
|
||||
* be configured to worst case in a healthy cluster.
|
||||
* @param durableStoreProps Props for the durable store actor,
|
||||
* the `Left` alternative is a tuple of fully qualified actor class name and
|
||||
* the config constructor parameter of that class,
|
||||
* the `Right` alternative is the `Props` of the actor.
|
||||
* @param durableKeys Keys that are durable. Prefix matching is supported by using
|
||||
* `*` at the end of a key. All entries can be made durable by including "*"
|
||||
* in the `Set`.
|
||||
*/
|
||||
final class ReplicatorSettings(
|
||||
val role: Option[String],
|
||||
|
|
@ -99,7 +117,15 @@ final class ReplicatorSettings(
|
|||
val maxDeltaElements: Int,
|
||||
val dispatcher: String,
|
||||
val pruningInterval: FiniteDuration,
|
||||
val maxPruningDissemination: FiniteDuration) {
|
||||
val maxPruningDissemination: FiniteDuration,
|
||||
val durableStoreProps: Either[(String, Config), Props],
|
||||
val durableKeys: Set[String]) {
|
||||
|
||||
// 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)
|
||||
|
||||
def withRole(role: String): ReplicatorSettings = copy(role = ReplicatorSettings.roleOption(role))
|
||||
|
||||
|
|
@ -125,16 +151,35 @@ final class ReplicatorSettings(
|
|||
def withPruning(pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration): ReplicatorSettings =
|
||||
copy(pruningInterval = pruningInterval, maxPruningDissemination = maxPruningDissemination)
|
||||
|
||||
def withDurableStoreProps(durableStoreProps: Props): ReplicatorSettings =
|
||||
copy(durableStoreProps = Right(durableStoreProps))
|
||||
|
||||
/**
|
||||
* Scala API
|
||||
*/
|
||||
def withDurableKeys(durableKeys: Set[String]): ReplicatorSettings =
|
||||
copy(durableKeys = durableKeys)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def withDurableKeys(durableKeys: java.util.Set[String]): ReplicatorSettings = {
|
||||
import scala.collection.JavaConverters._
|
||||
withDurableKeys(durableKeys.asScala.toSet)
|
||||
}
|
||||
|
||||
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): 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): ReplicatorSettings =
|
||||
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
|
||||
pruningInterval, maxPruningDissemination)
|
||||
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys)
|
||||
}
|
||||
|
||||
object Replicator {
|
||||
|
|
@ -142,8 +187,12 @@ object Replicator {
|
|||
/**
|
||||
* Factory method for the [[akka.actor.Props]] of the [[Replicator]] actor.
|
||||
*/
|
||||
def props(settings: ReplicatorSettings): Props =
|
||||
def props(settings: ReplicatorSettings): Props = {
|
||||
require(
|
||||
settings.durableKeys.isEmpty || (settings.durableStoreProps != Right(Props.empty)),
|
||||
"durableStoreProps must be defined when durableKeys are defined")
|
||||
Props(new Replicator(settings)).withDeploy(Deploy.local).withDispatcher(settings.dispatcher)
|
||||
}
|
||||
|
||||
sealed trait ReadConsistency {
|
||||
def timeout: FiniteDuration
|
||||
|
|
@ -400,6 +449,17 @@ object Replicator {
|
|||
extends UpdateFailure[A] {
|
||||
override def toString: String = s"ModifyFailure [$key]: $errorMessage"
|
||||
}
|
||||
/**
|
||||
* The local store or direct replication of the [[Update]] could not be fulfill according to
|
||||
* the given [[WriteConsistency consistency level]] due to durable store errors. This is
|
||||
* only used for entries that have been configured to be durable.
|
||||
*
|
||||
* The `Update` was still performed in memory locally and possibly replicated to some nodes,
|
||||
* but it might not have been written to durable storage.
|
||||
* It will eventually be disseminated to other replicas, unless the local replica
|
||||
* crashes before it has been able to communicate with other replicas.
|
||||
*/
|
||||
final case class StoreFailure[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends UpdateFailure[A] with DeleteResponse[A]
|
||||
|
||||
/**
|
||||
* Send this message to the local `Replicator` to delete a data value for the
|
||||
|
|
@ -460,6 +520,7 @@ object Replicator {
|
|||
case object ClockTick
|
||||
final case class Write(key: String, envelope: DataEnvelope) extends ReplicatorMessage
|
||||
case object WriteAck extends ReplicatorMessage with DeadLetterSuppression
|
||||
case object WriteNack extends ReplicatorMessage with DeadLetterSuppression
|
||||
final case class Read(key: String) extends ReplicatorMessage
|
||||
final case class ReadResult(envelope: Option[DataEnvelope]) extends ReplicatorMessage with DeadLetterSuppression
|
||||
final case class ReadRepair(key: String, envelope: DataEnvelope)
|
||||
|
|
@ -507,7 +568,8 @@ object Replicator {
|
|||
var mergedRemovedNodePruning = other.pruning
|
||||
for ((key, thisValue) ← pruning) {
|
||||
mergedRemovedNodePruning.get(key) match {
|
||||
case None ⇒ mergedRemovedNodePruning = mergedRemovedNodePruning.updated(key, thisValue)
|
||||
case None ⇒
|
||||
mergedRemovedNodePruning = mergedRemovedNodePruning.updated(key, thisValue)
|
||||
case Some(thatValue) ⇒
|
||||
mergedRemovedNodePruning = mergedRemovedNodePruning.updated(key, thisValue merge thatValue)
|
||||
}
|
||||
|
|
@ -751,6 +813,21 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
val serializer = SerializationExtension(context.system).serializerFor(classOf[DataEnvelope])
|
||||
val maxPruningDisseminationNanos = maxPruningDissemination.toNanos
|
||||
|
||||
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 durableStore: ActorRef =
|
||||
if (hasDurableKeys) {
|
||||
val props = settings.durableStoreProps match {
|
||||
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"))
|
||||
}
|
||||
context.watch(context.actorOf(props.withDeploy(Deploy.local), "durableStore"))
|
||||
} else
|
||||
context.system.deadLetters // not used
|
||||
|
||||
// cluster nodes, doesn't contain selfAddress
|
||||
var nodes: Set[Address] = Set.empty
|
||||
|
||||
|
|
@ -784,6 +861,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
var subscriptionKeys = Map.empty[String, KeyR]
|
||||
|
||||
override def preStart(): Unit = {
|
||||
if (hasDurableKeys)
|
||||
durableStore ! LoadAll
|
||||
val leaderChangedClass = if (role.isDefined) classOf[RoleLeaderChanged] else classOf[LeaderChanged]
|
||||
cluster.subscribe(self, initialStateMode = InitialStateAsEvents,
|
||||
classOf[MemberEvent], classOf[ReachabilityEvent], leaderChangedClass)
|
||||
|
|
@ -799,7 +878,47 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
def matchingRole(m: Member): Boolean = role.forall(m.hasRole)
|
||||
|
||||
def receive = normalReceive
|
||||
override val supervisorStrategy = {
|
||||
def fromDurableStore: Boolean = sender() == durableStore && sender() != context.system.deadLetters
|
||||
OneForOneStrategy()(
|
||||
({
|
||||
case e @ (_: DurableStore.LoadFailed | _: ActorInitializationException) if fromDurableStore ⇒
|
||||
log.error(e, "Stopping distributed-data Replicator due to load or startup failure in durable store")
|
||||
context.stop(self)
|
||||
SupervisorStrategy.Stop
|
||||
}: SupervisorStrategy.Decider).orElse(SupervisorStrategy.defaultDecider))
|
||||
}
|
||||
|
||||
def receive =
|
||||
if (hasDurableKeys) load.orElse(normalReceive)
|
||||
else normalReceive
|
||||
|
||||
val load: Receive = {
|
||||
case LoadData(data) ⇒
|
||||
data.foreach {
|
||||
case (key, d) ⇒
|
||||
val envelope = DataEnvelope(d)
|
||||
write(key, envelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (newEnvelope.data ne envelope.data)
|
||||
durableStore ! Store(key, newEnvelope.data, None)
|
||||
case None ⇒
|
||||
}
|
||||
}
|
||||
case LoadAllCompleted ⇒
|
||||
context.become(normalReceive)
|
||||
self ! FlushChanges
|
||||
|
||||
case GetReplicaCount ⇒
|
||||
// 0 until durable data has been loaded, used by test
|
||||
sender() ! ReplicaCount(0)
|
||||
|
||||
case RemovedNodePruningTick | FlushChanges | GossipTick ⇒
|
||||
// ignore scheduled ticks when loading durable data
|
||||
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)
|
||||
}
|
||||
|
||||
val normalReceive: Receive = {
|
||||
case Get(key, consistency, req) ⇒ receiveGet(key, consistency, req)
|
||||
|
|
@ -872,11 +991,22 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
log.debug("Received Update for key [{}], old data [{}], new data [{}]", key, localValue, newData)
|
||||
val envelope = DataEnvelope(pruningCleanupTombstoned(newData))
|
||||
setData(key.id, envelope)
|
||||
if (isLocalUpdate(writeConsistency))
|
||||
sender() ! UpdateSuccess(key, req)
|
||||
else
|
||||
context.actorOf(WriteAggregator.props(key, envelope, writeConsistency, req, nodes, sender())
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
val durable = isDurable(key.id)
|
||||
if (isLocalUpdate(writeConsistency)) {
|
||||
if (durable)
|
||||
durableStore ! Store(key.id, envelope.data,
|
||||
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), sender())))
|
||||
else
|
||||
sender() ! UpdateSuccess(key, req)
|
||||
} else {
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, envelope, writeConsistency, req, nodes, sender(), durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, envelope.data,
|
||||
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), writeAggregator)))
|
||||
}
|
||||
}
|
||||
case Failure(e: DataDeleted[_]) ⇒
|
||||
log.debug("Received Update for deleted key [{}]", key)
|
||||
sender() ! e
|
||||
|
|
@ -886,6 +1016,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
}
|
||||
|
||||
def isDurable(key: String): Boolean =
|
||||
durable(key) || (durableWildcards.nonEmpty && durableWildcards.exists(key.startsWith))
|
||||
|
||||
def isLocalUpdate(writeConsistency: WriteConsistency): Boolean =
|
||||
writeConsistency match {
|
||||
case WriteLocal ⇒ true
|
||||
|
|
@ -894,28 +1027,43 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
def receiveWrite(key: String, envelope: DataEnvelope): Unit = {
|
||||
write(key, envelope)
|
||||
sender() ! WriteAck
|
||||
write(key, envelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, newEnvelope.data, Some(StoreReply(WriteAck, WriteNack, sender())))
|
||||
else
|
||||
sender() ! WriteAck
|
||||
case None ⇒
|
||||
}
|
||||
}
|
||||
|
||||
def write(key: String, writeEnvelope: DataEnvelope): Unit =
|
||||
def write(key: String, writeEnvelope: DataEnvelope): Option[DataEnvelope] =
|
||||
getData(key) match {
|
||||
case Some(DataEnvelope(DeletedData, _)) ⇒ // already deleted
|
||||
case Some(DataEnvelope(DeletedData, _)) ⇒ Some(writeEnvelope) // already deleted
|
||||
case Some(envelope @ DataEnvelope(existing, _)) ⇒
|
||||
if (existing.getClass == writeEnvelope.data.getClass || writeEnvelope.data == DeletedData) {
|
||||
val merged = envelope.merge(pruningCleanupTombstoned(writeEnvelope)).addSeen(selfAddress)
|
||||
setData(key, merged)
|
||||
Some(merged)
|
||||
} else {
|
||||
log.warning(
|
||||
"Wrong type for writing [{}], existing type [{}], got [{}]",
|
||||
key, existing.getClass.getName, writeEnvelope.data.getClass.getName)
|
||||
None
|
||||
}
|
||||
case None ⇒
|
||||
setData(key, pruningCleanupTombstoned(writeEnvelope).addSeen(selfAddress))
|
||||
val cleaned = pruningCleanupTombstoned(writeEnvelope).addSeen(selfAddress)
|
||||
setData(key, cleaned)
|
||||
Some(cleaned)
|
||||
}
|
||||
|
||||
def receiveReadRepair(key: String, writeEnvelope: DataEnvelope): Unit = {
|
||||
write(key, writeEnvelope)
|
||||
write(key, writeEnvelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, newEnvelope.data, None)
|
||||
case None ⇒
|
||||
}
|
||||
sender() ! ReadRepairAck
|
||||
}
|
||||
|
||||
|
|
@ -933,11 +1081,22 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
sender() ! DataDeleted(key)
|
||||
case _ ⇒
|
||||
setData(key.id, DeletedEnvelope)
|
||||
if (isLocalUpdate(consistency))
|
||||
sender() ! DeleteSuccess(key)
|
||||
else
|
||||
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, None, nodes, sender())
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
val durable = isDurable(key.id)
|
||||
if (isLocalUpdate(consistency)) {
|
||||
if (durable)
|
||||
durableStore ! Store(key.id, DeletedData,
|
||||
Some(StoreReply(DeleteSuccess(key), StoreFailure(key, None), sender())))
|
||||
else
|
||||
sender() ! DeleteSuccess(key)
|
||||
} else {
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, None, nodes, sender(), durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, DeletedData,
|
||||
Some(StoreReply(DeleteSuccess(key), StoreFailure(key, None), writeAggregator)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1075,7 +1234,12 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
updatedData.foreach {
|
||||
case (key, envelope) ⇒
|
||||
val hadData = dataEntries.contains(key)
|
||||
write(key, envelope)
|
||||
write(key, envelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, newEnvelope.data, None)
|
||||
case None ⇒
|
||||
}
|
||||
if (sendBack) getData(key) match {
|
||||
case Some(d) ⇒
|
||||
if (hadData || d.pruning.nonEmpty)
|
||||
|
|
@ -1108,14 +1272,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
(newSubscribers.exists { case (k, s) ⇒ s.contains(subscriber) })
|
||||
|
||||
def receiveTerminated(ref: ActorRef): Unit = {
|
||||
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) }
|
||||
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) }
|
||||
|
||||
(keys1 ++ keys2).foreach { key ⇒
|
||||
if (!subscribers.contains(key) && !newSubscribers.contains(key))
|
||||
subscriptionKeys -= key
|
||||
(keys1 ++ keys2).foreach { key ⇒
|
||||
if (!subscribers.contains(key) && !newSubscribers.contains(key))
|
||||
subscriptionKeys -= key
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1161,7 +1330,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
initRemovedNodePruning()
|
||||
}
|
||||
performRemovedNodePruning()
|
||||
tombstoneRemovedNodePruning()
|
||||
// FIXME tombstoneRemovedNodePruning doesn't work, since merge of PruningState will add the PruningPerformed back again
|
||||
// tombstoneRemovedNodePruning()
|
||||
}
|
||||
|
||||
def initRemovedNodePruning(): Unit = {
|
||||
|
|
@ -1171,6 +1341,7 @@ 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 = {
|
||||
|
|
@ -1206,6 +1377,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
pruningPerformed = pruningPerformed.updated(removed, allReachableClockTime)
|
||||
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)
|
||||
case _ ⇒
|
||||
}
|
||||
case _ ⇒ // deleted, or pruning not needed
|
||||
|
|
@ -1225,6 +1398,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
}
|
||||
|
||||
// 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) ⇒
|
||||
|
|
@ -1234,7 +1408,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
tombstoneNodes += removed
|
||||
dataEntries.foreach {
|
||||
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, _), _)) ⇒
|
||||
setData(key, pruningCleanupTombstoned(removed, envelope))
|
||||
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
|
||||
}
|
||||
case (removed, timestamp) ⇒ // not ready
|
||||
|
|
@ -1325,8 +1502,9 @@ private[akka] object WriteAggregator {
|
|||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
nodes: Set[Address],
|
||||
replyTo: ActorRef): Props =
|
||||
Props(new WriteAggregator(key, envelope, consistency, req, nodes, replyTo))
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean): Props =
|
||||
Props(new WriteAggregator(key, envelope, consistency, req, nodes, replyTo, durable))
|
||||
.withDeploy(Deploy.local)
|
||||
}
|
||||
|
||||
|
|
@ -1339,7 +1517,8 @@ private[akka] class WriteAggregator(
|
|||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
replyTo: ActorRef) extends ReadWriteAggregator {
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean) extends ReadWriteAggregator {
|
||||
|
||||
import Replicator._
|
||||
import Replicator.Internal._
|
||||
|
|
@ -1355,41 +1534,65 @@ private[akka] class WriteAggregator(
|
|||
val w = N / 2 + 1 // write to at least (N/2+1) nodes
|
||||
N - w
|
||||
case WriteLocal ⇒
|
||||
throw new IllegalArgumentException("ReadLocal not supported by WriteAggregator")
|
||||
throw new IllegalArgumentException("WriteLocal not supported by WriteAggregator")
|
||||
}
|
||||
|
||||
val writeMsg = Write(key.id, envelope)
|
||||
|
||||
var gotLocalStoreReply = !durable
|
||||
var gotWriteNackFrom = Set.empty[Address]
|
||||
|
||||
override def preStart(): Unit = {
|
||||
primaryNodes.foreach { replica(_) ! writeMsg }
|
||||
|
||||
if (remaining.size == doneWhenRemainingSize)
|
||||
reply(ok = true)
|
||||
else if (doneWhenRemainingSize < 0 || remaining.size < doneWhenRemainingSize)
|
||||
reply(ok = false)
|
||||
if (isDone) reply(isTimeout = false)
|
||||
}
|
||||
|
||||
def receive = {
|
||||
def receive: Receive = {
|
||||
case WriteAck ⇒
|
||||
remaining -= senderAddress()
|
||||
if (remaining.size == doneWhenRemainingSize)
|
||||
reply(ok = true)
|
||||
if (isDone) reply(isTimeout = false)
|
||||
case WriteNack ⇒
|
||||
gotWriteNackFrom += senderAddress()
|
||||
if (isDone) reply(isTimeout = false)
|
||||
|
||||
case _: Replicator.UpdateSuccess[_] ⇒
|
||||
gotLocalStoreReply = true
|
||||
if (isDone) reply(isTimeout = false)
|
||||
case f: Replicator.StoreFailure[_] ⇒
|
||||
gotLocalStoreReply = true
|
||||
gotWriteNackFrom += Cluster(context.system).selfAddress
|
||||
if (isDone) reply(isTimeout = false)
|
||||
|
||||
case SendToSecondary ⇒
|
||||
secondaryNodes.foreach { replica(_) ! writeMsg }
|
||||
case ReceiveTimeout ⇒ reply(ok = false)
|
||||
case ReceiveTimeout ⇒
|
||||
reply(isTimeout = true)
|
||||
}
|
||||
|
||||
def senderAddress(): Address = sender().path.address
|
||||
|
||||
def reply(ok: Boolean): Unit = {
|
||||
if (ok && envelope.data == DeletedData)
|
||||
replyTo.tell(DeleteSuccess(key), context.parent)
|
||||
else if (ok)
|
||||
replyTo.tell(UpdateSuccess(key, req), context.parent)
|
||||
else if (envelope.data == DeletedData)
|
||||
replyTo.tell(ReplicationDeleteFailure(key), context.parent)
|
||||
else
|
||||
replyTo.tell(UpdateTimeout(key, req), context.parent)
|
||||
def isDone: Boolean =
|
||||
gotLocalStoreReply &&
|
||||
(remaining.size <= doneWhenRemainingSize || (remaining diff gotWriteNackFrom).isEmpty ||
|
||||
notEnoughNodes)
|
||||
|
||||
def notEnoughNodes: Boolean =
|
||||
doneWhenRemainingSize < 0 || nodes.size < doneWhenRemainingSize
|
||||
|
||||
def reply(isTimeout: Boolean): Unit = {
|
||||
val isDelete = envelope.data == DeletedData
|
||||
val isSuccess = remaining.size <= doneWhenRemainingSize && !notEnoughNodes
|
||||
val isTimeoutOrNotEnoughNodes = isTimeout || notEnoughNodes || gotWriteNackFrom.isEmpty
|
||||
|
||||
val replyMsg =
|
||||
if (isSuccess && isDelete) DeleteSuccess(key)
|
||||
else if (isSuccess) UpdateSuccess(key, req)
|
||||
else if (isTimeoutOrNotEnoughNodes && isDelete) ReplicationDeleteFailure(key)
|
||||
else if (isTimeoutOrNotEnoughNodes) UpdateTimeout(key, req)
|
||||
else StoreFailure(key, req)
|
||||
|
||||
replyTo.tell(replyMsg, context.parent)
|
||||
context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue