Use delta in direct write also, #22188
* Follow up on the causal delivery of deltas. * The first implementation used full state for the direct Write messages, i.e. updates with WriteConsistency != LocalWrite * This is an optimization so that delatas are tried first and if they can't be applied it falls back to full state. * For simultanious updates the messages may be reordered because we create separate WriteAggregator actor and such, but normally they will be sent in order so the deltas will typically be received in order, otherwise we fall back to retrying with full state in the second round in the WriteAggregator.
This commit is contained in:
parent
b2759ab56a
commit
233e784154
9 changed files with 365 additions and 85 deletions
|
|
@ -743,7 +743,8 @@ object Replicator {
|
|||
final case class Gossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean) extends ReplicatorMessage
|
||||
|
||||
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long)
|
||||
final case class DeltaPropagation(fromNode: UniqueAddress, deltas: Map[KeyId, Delta]) extends ReplicatorMessage
|
||||
final case class DeltaPropagation(fromNode: UniqueAddress, reply: Boolean, deltas: Map[KeyId, Delta]) extends ReplicatorMessage
|
||||
case object DeltaNack extends ReplicatorMessage with DeadLetterSuppression
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -990,7 +991,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = {
|
||||
// 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, deltas.map {
|
||||
DeltaPropagation(selfUniqueAddress, reply = false, deltas.map {
|
||||
case (key, (d, fromSeqNr, toSeqNr)) ⇒ getData(key) match {
|
||||
case Some(envelope) ⇒ key → Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
|
||||
case None ⇒ key → Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||
|
|
@ -1149,7 +1150,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case Read(key) ⇒ receiveRead(key)
|
||||
case Write(key, envelope) ⇒ receiveWrite(key, envelope)
|
||||
case ReadRepair(key, envelope) ⇒ receiveReadRepair(key, envelope)
|
||||
case DeltaPropagation(from, deltas) ⇒ receiveDeltaPropagation(from, deltas)
|
||||
case DeltaPropagation(from, reply, deltas) ⇒ receiveDeltaPropagation(from, reply, deltas)
|
||||
case FlushChanges ⇒ receiveFlushChanges()
|
||||
case DeltaPropagationTick ⇒ receiveDeltaPropagationTick()
|
||||
case GossipTick ⇒ receiveGossipTick()
|
||||
|
|
@ -1242,13 +1243,16 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
else
|
||||
replyTo ! UpdateSuccess(key, req)
|
||||
} else {
|
||||
val writeEnvelope = delta match {
|
||||
case Some(d: RequiresCausalDeliveryOfDeltas) ⇒ newEnvelope
|
||||
case Some(d) ⇒ DataEnvelope(d)
|
||||
case None ⇒ newEnvelope
|
||||
val (writeEnvelope, writeDelta) = delta match {
|
||||
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)
|
||||
}
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, writeEnvelope, writeConsistency, req, nodes, unreachable, replyTo, durable)
|
||||
context.actorOf(WriteAggregator.props(key, writeEnvelope, writeDelta, writeConsistency,
|
||||
req, nodes, unreachable, replyTo, durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, new DurableDataEnvelope(newEnvelope),
|
||||
|
|
@ -1274,14 +1278,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case _ ⇒ false
|
||||
}
|
||||
|
||||
def receiveWrite(key: KeyId, envelope: DataEnvelope): Unit = {
|
||||
write(key, envelope) match {
|
||||
def receiveWrite(key: KeyId, envelope: DataEnvelope): Unit =
|
||||
writeAndStore(key, envelope, reply = true)
|
||||
|
||||
def writeAndStore(key: KeyId, writeEnvelope: DataEnvelope, reply: Boolean): Unit = {
|
||||
write(key, writeEnvelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), Some(StoreReply(WriteAck, WriteNack, replyTo)))
|
||||
else
|
||||
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 ⇒
|
||||
if (reply)
|
||||
replyTo ! WriteNack
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1316,17 +1326,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
}
|
||||
|
||||
def writeAndStore(key: KeyId, writeEnvelope: DataEnvelope): Unit = {
|
||||
write(key, writeEnvelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
|
||||
case None ⇒
|
||||
}
|
||||
}
|
||||
|
||||
def receiveReadRepair(key: KeyId, writeEnvelope: DataEnvelope): Unit = {
|
||||
writeAndStore(key, writeEnvelope)
|
||||
writeAndStore(key, writeEnvelope, reply = false)
|
||||
replyTo ! ReadRepairAck
|
||||
}
|
||||
|
||||
|
|
@ -1353,7 +1354,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
replyTo ! DeleteSuccess(key, req)
|
||||
} else {
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, req, nodes, unreachable, replyTo, durable)
|
||||
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, None, consistency, req, nodes, unreachable, replyTo, durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, new DurableDataEnvelope(DeletedEnvelope),
|
||||
|
|
@ -1464,7 +1465,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
deltaPropagationSelector.cleanupDeltaEntries()
|
||||
}
|
||||
|
||||
def receiveDeltaPropagation(fromNode: UniqueAddress, deltas: Map[KeyId, Delta]): Unit =
|
||||
def receiveDeltaPropagation(fromNode: UniqueAddress, reply: Boolean, deltas: Map[KeyId, Delta]): Unit =
|
||||
if (deltaCrdtEnabled) {
|
||||
try {
|
||||
val isDebugEnabled = log.isDebugEnabled
|
||||
|
|
@ -1485,20 +1486,22 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
if (isDebugEnabled) log.debug(
|
||||
"Skipping DeltaPropagation from [{}] for [{}] because toSeqNr [{}] already handled [{}]",
|
||||
fromNode.address, key, toSeqNr, currentSeqNr)
|
||||
if (reply) replyTo ! WriteAck
|
||||
} else if (fromSeqNr > (currentSeqNr + 1)) {
|
||||
if (isDebugEnabled) log.debug(
|
||||
"Skipping DeltaPropagation from [{}] for [{}] because missing deltas between [{}-{}]",
|
||||
fromNode.address, key, currentSeqNr + 1, fromSeqNr - 1)
|
||||
if (reply) replyTo ! DeltaNack
|
||||
} else {
|
||||
if (isDebugEnabled) log.debug(
|
||||
"Applying DeltaPropagation from [{}] for [{}] with sequence numbers [{}], current was [{}]",
|
||||
fromNode.address, key, s"$fromSeqNr-$toSeqNr", currentSeqNr)
|
||||
val newEnvelope = envelope.copy(deltaVersions = VersionVector(fromNode, toSeqNr))
|
||||
writeAndStore(key, newEnvelope)
|
||||
writeAndStore(key, newEnvelope, reply)
|
||||
}
|
||||
case (key, Delta(envelope, _, _)) ⇒
|
||||
// causal delivery of deltas not needed, just apply it
|
||||
writeAndStore(key, envelope)
|
||||
writeAndStore(key, envelope, reply)
|
||||
}
|
||||
}
|
||||
} catch {
|
||||
|
|
@ -1507,6 +1510,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
// mixing nodes with incompatible delta-CRDT types
|
||||
log.warning("Couldn't process DeltaPropagation from [] due to {}", fromNode, e)
|
||||
}
|
||||
} else {
|
||||
// !deltaCrdtEnabled
|
||||
if (reply) replyTo ! DeltaNack
|
||||
}
|
||||
|
||||
def receiveGossipTick(): Unit = {
|
||||
|
|
@ -1583,7 +1589,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
updatedData.foreach {
|
||||
case (key, envelope) ⇒
|
||||
val hadData = dataEntries.contains(key)
|
||||
writeAndStore(key, envelope)
|
||||
writeAndStore(key, envelope, reply = false)
|
||||
if (sendBack) getData(key) match {
|
||||
case Some(d) ⇒
|
||||
if (hadData || d.pruning.nonEmpty)
|
||||
|
|
@ -1842,13 +1848,14 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
def props(
|
||||
key: KeyR,
|
||||
envelope: Replicator.Internal.DataEnvelope,
|
||||
delta: Option[Replicator.Internal.Delta],
|
||||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean): Props =
|
||||
Props(new WriteAggregator(key, envelope, consistency, req, nodes, unreachable, replyTo, durable))
|
||||
Props(new WriteAggregator(key, envelope, delta, consistency, req, nodes, unreachable, replyTo, durable))
|
||||
.withDeploy(Deploy.local)
|
||||
}
|
||||
|
||||
|
|
@ -1858,6 +1865,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
@InternalApi private[akka] class WriteAggregator(
|
||||
key: KeyR,
|
||||
envelope: Replicator.Internal.DataEnvelope,
|
||||
delta: Option[Replicator.Internal.Delta],
|
||||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
|
|
@ -1869,6 +1877,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
import Replicator.Internal._
|
||||
import ReadWriteAggregator._
|
||||
|
||||
val selfUniqueAddress = Cluster(context.system).selfUniqueAddress
|
||||
|
||||
override def timeout: FiniteDuration = consistency.timeout
|
||||
|
||||
override val doneWhenRemainingSize = consistency match {
|
||||
|
|
@ -1883,12 +1893,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
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)))
|
||||
}
|
||||
|
||||
var gotLocalStoreReply = !durable
|
||||
var gotWriteNackFrom = Set.empty[Address]
|
||||
|
||||
override def preStart(): Unit = {
|
||||
primaryNodes.foreach { replica(_) ! writeMsg }
|
||||
val msg = deltaMsg match {
|
||||
case Some(d) ⇒ d
|
||||
case None ⇒ writeMsg
|
||||
}
|
||||
primaryNodes.foreach { replica(_) ! msg }
|
||||
|
||||
if (isDone) reply(isTimeout = false)
|
||||
}
|
||||
|
|
@ -1900,16 +1918,26 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case WriteNack ⇒
|
||||
gotWriteNackFrom += senderAddress()
|
||||
if (isDone) reply(isTimeout = false)
|
||||
case DeltaNack ⇒
|
||||
// ok, will be retried with full state
|
||||
|
||||
case _: Replicator.UpdateSuccess[_] ⇒
|
||||
gotLocalStoreReply = true
|
||||
if (isDone) reply(isTimeout = false)
|
||||
case f: Replicator.StoreFailure[_] ⇒
|
||||
gotLocalStoreReply = true
|
||||
gotWriteNackFrom += Cluster(context.system).selfAddress
|
||||
gotWriteNackFrom += selfUniqueAddress.address
|
||||
if (isDone) reply(isTimeout = false)
|
||||
|
||||
case SendToSecondary ⇒
|
||||
deltaMsg match {
|
||||
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 ⇒
|
||||
reply(isTimeout = true)
|
||||
|
|
@ -1934,7 +1962,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
if (isSuccess && isDelete) DeleteSuccess(key, req)
|
||||
else if (isSuccess) UpdateSuccess(key, req)
|
||||
else if (isTimeoutOrNotEnoughNodes && isDelete) ReplicationDeleteFailure(key, req)
|
||||
else if (isTimeoutOrNotEnoughNodes) UpdateTimeout(key, req)
|
||||
else if (isTimeoutOrNotEnoughNodes || !durable) UpdateTimeout(key, req)
|
||||
else StoreFailure(key, req)
|
||||
|
||||
replyTo.tell(replyMsg, context.parent)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue