discard large deltas, #23025
* to avoid OversizedPayloadException * some complex deltas grow for each update operation, e.g. when updating different keys in ORMap (PNCounterMap) * such large deltas can safely be discarded and disseminated as full state instead * added ReplicatedDeltaSize interface to be able to define the "size" and when that size exceeds configured threshold the delta is discarded
This commit is contained in:
parent
5641c12547
commit
2970287f95
9 changed files with 117 additions and 13 deletions
|
|
@ -62,6 +62,11 @@ akka.cluster.distributed-data {
|
|||
delta-crdt {
|
||||
# enable or disable delta-CRDT replication
|
||||
enabled = on
|
||||
|
||||
# Some complex deltas grow in size for each update and above this
|
||||
# threshold such deltas are discarded and sent as full state instead.
|
||||
# This is number of elements or similar size hint, not size in bytes.
|
||||
max-delta-size = 200
|
||||
}
|
||||
|
||||
durable {
|
||||
|
|
|
|||
|
|
@ -30,6 +30,8 @@ import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholde
|
|||
|
||||
def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation
|
||||
|
||||
def maxDeltaSize: Int
|
||||
|
||||
def currentVersion(key: KeyId): Long = deltaCounter.get(key) match {
|
||||
case Some(v) ⇒ v
|
||||
case None ⇒ 0L
|
||||
|
|
@ -106,12 +108,18 @@ import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholde
|
|||
case None ⇒
|
||||
val group = deltaEntriesAfterJ.valuesIterator.reduceLeft {
|
||||
(d1, d2) ⇒
|
||||
d2 match {
|
||||
val merged = d2 match {
|
||||
case NoDeltaPlaceholder ⇒ NoDeltaPlaceholder
|
||||
case _ ⇒
|
||||
// this is fine also if d1 is a NoDeltaPlaceholder
|
||||
d1.merge(d2.asInstanceOf[d1.T])
|
||||
}
|
||||
merged match {
|
||||
case s: ReplicatedDeltaSize if s.deltaSize >= maxDeltaSize ⇒
|
||||
// discard too large deltas
|
||||
NoDeltaPlaceholder
|
||||
case _ ⇒ merged
|
||||
}
|
||||
}
|
||||
cache = cache.updated(cacheKey, group)
|
||||
group
|
||||
|
|
|
|||
|
|
@ -51,7 +51,7 @@ object ORMap {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] sealed abstract class AtomicDeltaOp[A, B <: ReplicatedData] extends DeltaOp {
|
||||
@InternalApi private[akka] sealed abstract class AtomicDeltaOp[A, B <: ReplicatedData] extends DeltaOp with ReplicatedDeltaSize {
|
||||
def underlying: ORSet.DeltaOp
|
||||
def zeroTag: ZeroTag
|
||||
override def zero: DeltaReplicatedData = zeroTag.zero
|
||||
|
|
@ -59,6 +59,7 @@ object ORMap {
|
|||
case other: AtomicDeltaOp[A, B] ⇒ DeltaGroup(Vector(this, other))
|
||||
case DeltaGroup(ops) ⇒ DeltaGroup(this +: ops)
|
||||
}
|
||||
override def deltaSize: Int = 1
|
||||
}
|
||||
|
||||
// PutDeltaOp contains ORSet delta and full value
|
||||
|
|
@ -117,7 +118,8 @@ object ORMap {
|
|||
|
||||
// DeltaGroup is effectively a causally ordered list of individual deltas
|
||||
/** INTERNAL API */
|
||||
@InternalApi private[akka] final case class DeltaGroup[A, B <: ReplicatedData](ops: immutable.IndexedSeq[DeltaOp]) extends DeltaOp {
|
||||
@InternalApi private[akka] final case class DeltaGroup[A, B <: ReplicatedData](ops: immutable.IndexedSeq[DeltaOp])
|
||||
extends DeltaOp with ReplicatedDeltaSize {
|
||||
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||
case that: AtomicDeltaOp[A, B] ⇒
|
||||
ops.last match {
|
||||
|
|
@ -139,6 +141,8 @@ object ORMap {
|
|||
}
|
||||
|
||||
override def zero: DeltaReplicatedData = ops.headOption.fold(ORMap.empty[A, B].asInstanceOf[DeltaReplicatedData])(_.zero)
|
||||
|
||||
override def deltaSize: Int = ops.size
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -45,9 +45,10 @@ object ORSet {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] sealed abstract class AtomicDeltaOp[A] extends DeltaOp {
|
||||
@InternalApi private[akka] sealed abstract class AtomicDeltaOp[A] extends DeltaOp with ReplicatedDeltaSize {
|
||||
def underlying: ORSet[A]
|
||||
override def zero: ORSet[A] = ORSet.empty
|
||||
override def deltaSize: Int = 1
|
||||
}
|
||||
|
||||
/** INTERNAL API */
|
||||
|
|
@ -94,7 +95,8 @@ object ORSet {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final case class DeltaGroup[A](ops: immutable.IndexedSeq[DeltaOp]) extends DeltaOp {
|
||||
@InternalApi private[akka] final case class DeltaGroup[A](ops: immutable.IndexedSeq[DeltaOp])
|
||||
extends DeltaOp with ReplicatedDeltaSize {
|
||||
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||
case thatAdd: AddDeltaOp[A] ⇒
|
||||
// merge AddDeltaOp into last AddDeltaOp in the group, if possible
|
||||
|
|
@ -107,6 +109,8 @@ object ORSet {
|
|||
}
|
||||
|
||||
override def zero: ORSet[A] = ORSet.empty
|
||||
|
||||
override def deltaSize: Int = ops.size
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -114,6 +114,18 @@ trait ReplicatedDelta extends ReplicatedData {
|
|||
*/
|
||||
trait RequiresCausalDeliveryOfDeltas extends ReplicatedDelta
|
||||
|
||||
/**
|
||||
* Some complex deltas grow in size for each update and above a configured
|
||||
* threshold such deltas are discarded and sent as full state instead. This
|
||||
* interface should be implemented by such deltas to define its size.
|
||||
* This is number of elements or similar size hint, not size in bytes.
|
||||
* The threshold is defined in `akka.cluster.distributed-data.delta-crdt.max-delta-size`
|
||||
* or corresponding [[ReplicatorSettings]].
|
||||
*/
|
||||
trait ReplicatedDeltaSize {
|
||||
def deltaSize: Int
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Interface for implementing a [[ReplicatedData]] in Java.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -86,7 +86,8 @@ object ReplicatorSettings {
|
|||
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,
|
||||
deltaCrdtEnabled = config.getBoolean("delta-crdt.enabled"))
|
||||
deltaCrdtEnabled = config.getBoolean("delta-crdt.enabled"),
|
||||
maxDeltaSize = config.getInt("delta-crdt.max-delta-size"))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -134,20 +135,31 @@ final class ReplicatorSettings(
|
|||
val durableKeys: Set[KeyId],
|
||||
val pruningMarkerTimeToLive: FiniteDuration,
|
||||
val durablePruningMarkerTimeToLive: FiniteDuration,
|
||||
val deltaCrdtEnabled: Boolean) {
|
||||
val deltaCrdtEnabled: Boolean,
|
||||
val maxDeltaSize: Int) {
|
||||
|
||||
// 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, 6.hours, 10.days, true)
|
||||
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days, true, 200)
|
||||
|
||||
// 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, true)
|
||||
maxPruningDissemination, durableStoreProps, durableKeys, 6.hours, 10.days, true, 200)
|
||||
|
||||
// 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],
|
||||
pruningMarkerTimeToLive: FiniteDuration, durablePruningMarkerTimeToLive: FiniteDuration,
|
||||
deltaCrdtEnabled: Boolean) =
|
||||
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
||||
maxPruningDissemination, durableStoreProps, durableKeys, pruningMarkerTimeToLive, durablePruningMarkerTimeToLive,
|
||||
deltaCrdtEnabled, 200)
|
||||
|
||||
def withRole(role: String): ReplicatorSettings = copy(role = ReplicatorSettings.roleOption(role))
|
||||
|
||||
|
|
@ -200,6 +212,9 @@ final class ReplicatorSettings(
|
|||
def withDeltaCrdtEnabled(deltaCrdtEnabled: Boolean): ReplicatorSettings =
|
||||
copy(deltaCrdtEnabled = deltaCrdtEnabled)
|
||||
|
||||
def withMaxDeltaSize(maxDeltaSize: Int): ReplicatorSettings =
|
||||
copy(maxDeltaSize = maxDeltaSize)
|
||||
|
||||
private def copy(
|
||||
role: Option[String] = role,
|
||||
gossipInterval: FiniteDuration = gossipInterval,
|
||||
|
|
@ -212,10 +227,11 @@ final class ReplicatorSettings(
|
|||
durableKeys: Set[KeyId] = durableKeys,
|
||||
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
|
||||
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
|
||||
deltaCrdtEnabled: Boolean = deltaCrdtEnabled): ReplicatorSettings =
|
||||
deltaCrdtEnabled: Boolean = deltaCrdtEnabled,
|
||||
maxDeltaSize: Int = maxDeltaSize): ReplicatorSettings =
|
||||
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
|
||||
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys,
|
||||
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive, deltaCrdtEnabled)
|
||||
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive, deltaCrdtEnabled, maxDeltaSize)
|
||||
}
|
||||
|
||||
object Replicator {
|
||||
|
|
@ -1010,6 +1026,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
nodes.union(weaklyUpNodes).diff(unreachable).toVector.sorted
|
||||
}
|
||||
|
||||
override def maxDeltaSize: Int = settings.maxDeltaSize
|
||||
|
||||
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.
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue