Merge pull request #22378 from akka/wip-delta-in-direct-write-patriknw
Use delta in direct write also, #22188
This commit is contained in:
commit
6b5b819c73
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