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:
Patrik Nordwall 2017-05-30 12:09:56 +02:00
parent 5641c12547
commit 2970287f95
9 changed files with 117 additions and 13 deletions

View file

@ -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 {

View file

@ -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

View file

@ -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
}
}

View file

@ -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
}
/**

View file

@ -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.
*

View file

@ -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.