Harden ShardCoordinator state replication, #28856 (#28895)

* Possibility to prefer oldest in ddata writes and reads
  * enabled for Cluster Sharding
* New ReadMajorityPlus and WriteMajorityPlus
  * used by Cluster Sharding, with configuration
  * also possible to define ReadAll in config
This commit is contained in:
Patrik Nordwall 2020-04-24 14:19:53 +02:00 committed by GitHub
parent d415211bf8
commit 4ba835d328
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
17 changed files with 767 additions and 196 deletions

View file

@ -64,7 +64,6 @@ import scala.annotation.varargs
import akka.event.Logging
import akka.util.JavaDurationConverters._
import akka.util.ccompat._
import com.github.ghik.silencer.silent
@ccompatUsedUntil213
@ -91,7 +90,7 @@ object ReplicatorSettings {
import akka.util.ccompat.JavaConverters._
new ReplicatorSettings(
role = roleOption(config.getString("role")),
roles = roleOption(config.getString("role")).toSet,
gossipInterval = config.getDuration("gossip-interval", MILLISECONDS).millis,
notifySubscribersInterval = config.getDuration("notify-subscribers-interval", MILLISECONDS).millis,
maxDeltaElements = config.getInt("max-delta-elements"),
@ -103,7 +102,8 @@ object ReplicatorSettings {
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"),
maxDeltaSize = config.getInt("delta-crdt.max-delta-size"))
maxDeltaSize = config.getInt("delta-crdt.max-delta-size"),
preferOldest = config.getBoolean("prefer-oldest"))
}
/**
@ -147,6 +147,7 @@ object ReplicatorSettings {
* @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`.
* @param preferOldest Update and Get operations are sent to oldest nodes first.
*/
final class ReplicatorSettings(
val roles: Set[String],
@ -161,7 +162,39 @@ final class ReplicatorSettings(
val pruningMarkerTimeToLive: FiniteDuration,
val durablePruningMarkerTimeToLive: FiniteDuration,
val deltaCrdtEnabled: Boolean,
val maxDeltaSize: Int) {
val maxDeltaSize: Int,
val preferOldest: Boolean) {
// for backwards compatibility
def this(
roles: Set[String],
gossipInterval: FiniteDuration,
notifySubscribersInterval: FiniteDuration,
maxDeltaElements: Int,
dispatcher: String,
pruningInterval: FiniteDuration,
maxPruningDissemination: FiniteDuration,
durableStoreProps: Either[(String, Config), Props],
durableKeys: Set[KeyId],
pruningMarkerTimeToLive: FiniteDuration,
durablePruningMarkerTimeToLive: FiniteDuration,
deltaCrdtEnabled: Boolean,
maxDeltaSize: Int) =
this(
roles,
gossipInterval,
notifySubscribersInterval,
maxDeltaElements,
dispatcher,
pruningInterval,
maxPruningDissemination,
durableStoreProps,
durableKeys,
pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive,
deltaCrdtEnabled,
maxDeltaSize,
preferOldest = false)
// for backwards compatibility
def this(
@ -179,7 +212,7 @@ final class ReplicatorSettings(
deltaCrdtEnabled: Boolean,
maxDeltaSize: Int) =
this(
role.iterator.toSet,
role.toSet,
gossipInterval,
notifySubscribersInterval,
maxDeltaElements,
@ -203,7 +236,7 @@ final class ReplicatorSettings(
pruningInterval: FiniteDuration,
maxPruningDissemination: FiniteDuration) =
this(
roles = role.iterator.toSet,
roles = role.toSet,
gossipInterval,
notifySubscribersInterval,
maxDeltaElements,
@ -272,9 +305,9 @@ final class ReplicatorSettings(
deltaCrdtEnabled,
200)
def withRole(role: String): ReplicatorSettings = copy(roles = ReplicatorSettings.roleOption(role).iterator.toSet)
def withRole(role: String): ReplicatorSettings = copy(roles = ReplicatorSettings.roleOption(role).toSet)
def withRole(role: Option[String]): ReplicatorSettings = copy(roles = role.iterator.toSet)
def withRole(role: Option[String]): ReplicatorSettings = copy(roles = role.toSet)
@varargs
def withRoles(roles: String*): ReplicatorSettings = copy(roles = roles.toSet)
@ -337,6 +370,9 @@ final class ReplicatorSettings(
def withMaxDeltaSize(maxDeltaSize: Int): ReplicatorSettings =
copy(maxDeltaSize = maxDeltaSize)
def withPreferOldest(preferOldest: Boolean): ReplicatorSettings =
copy(preferOldest = preferOldest)
private def copy(
roles: Set[String] = roles,
gossipInterval: FiniteDuration = gossipInterval,
@ -350,7 +386,8 @@ final class ReplicatorSettings(
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
deltaCrdtEnabled: Boolean = deltaCrdtEnabled,
maxDeltaSize: Int = maxDeltaSize): ReplicatorSettings =
maxDeltaSize: Int = maxDeltaSize,
preferOldest: Boolean = preferOldest): ReplicatorSettings =
new ReplicatorSettings(
roles,
gossipInterval,
@ -364,7 +401,8 @@ final class ReplicatorSettings(
pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive,
deltaCrdtEnabled,
maxDeltaSize)
maxDeltaSize,
preferOldest)
}
object Replicator {
@ -403,6 +441,19 @@ object Replicator {
*/
def this(timeout: java.time.Duration) = this(timeout.asScala, DefaultMajorityMinCap)
}
/**
* `ReadMajority` but with the given number of `additional` nodes added to the majority count. At most
* all nodes.
*/
final case class ReadMajorityPlus(timeout: FiniteDuration, additional: Int, minCap: Int = DefaultMajorityMinCap)
extends ReadConsistency {
/**
* Java API
*/
def this(timeout: java.time.Duration, additional: Int) = this(timeout.asScala, additional, DefaultMajorityMinCap)
}
final case class ReadAll(timeout: FiniteDuration) extends ReadConsistency {
/**
@ -434,6 +485,19 @@ object Replicator {
*/
def this(timeout: java.time.Duration) = this(timeout.asScala, DefaultMajorityMinCap)
}
/**
* `WriteMajority` but with the given number of `additional` nodes added to the majority count. At most
* all nodes.
*/
final case class WriteMajorityPlus(timeout: FiniteDuration, additional: Int, minCap: Int = DefaultMajorityMinCap)
extends WriteConsistency {
/**
* Java API
*/
def this(timeout: java.time.Duration, additional: Int) = this(timeout.asScala, additional, DefaultMajorityMinCap)
}
final case class WriteAll(timeout: FiniteDuration) extends WriteConsistency {
/**
@ -1008,7 +1072,9 @@ object Replicator {
extends ReplicatorMessage
with DestinationSystemUid
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long)
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long) {
def requiresCausalDeliveryOfDeltas: Boolean = dataEnvelope.data.isInstanceOf[RequiresCausalDeliveryOfDeltas]
}
final case class DeltaPropagation(_fromNode: UniqueAddress, reply: Boolean, deltas: Map[KeyId, Delta])
extends ReplicatorMessage
with SendingSystemUid {
@ -1288,8 +1354,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val deltaPropagationSelector = new DeltaPropagationSelector {
override val gossipIntervalDivisor = 5
override def allNodes: Vector[UniqueAddress] = {
// TODO optimize, by maintaining a sorted instance variable instead
Replicator.this.allNodes.diff(unreachable).toVector.sorted
// Replicator.allNodes is sorted
Replicator.this.allNodes.diff(unreachable).toVector
}
override def maxDeltaSize: Int = settings.maxDeltaSize
@ -1321,16 +1387,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
} else None
// cluster nodes, doesn't contain selfAddress, doesn't contain joining and weaklyUp
var nodes: Set[UniqueAddress] = Set.empty
var nodes: immutable.SortedSet[UniqueAddress] = immutable.SortedSet.empty
// cluster members sorted by age, oldest first,, doesn't contain selfAddress, doesn't contain joining and weaklyUp
// only used when prefer-oldest is enabled
var membersByAge: immutable.SortedSet[Member] = immutable.SortedSet.empty(Member.ageOrdering)
// cluster weaklyUp nodes, doesn't contain selfAddress
var weaklyUpNodes: Set[UniqueAddress] = Set.empty
var weaklyUpNodes: immutable.SortedSet[UniqueAddress] = immutable.SortedSet.empty
// cluster joining nodes, doesn't contain selfAddress
var joiningNodes: Set[UniqueAddress] = Set.empty
var joiningNodes: immutable.SortedSet[UniqueAddress] = immutable.SortedSet.empty
// up and weaklyUp nodes, doesn't contain joining and not selfAddress
private def allNodes: Set[UniqueAddress] = nodes.union(weaklyUpNodes)
private def allNodes: immutable.SortedSet[UniqueAddress] = nodes.union(weaklyUpNodes)
private def isKnownNode(node: UniqueAddress): Boolean =
nodes(node) || weaklyUpNodes(node) || joiningNodes(node) || selfUniqueAddress == node
@ -1370,6 +1440,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
// messages after loading durable data.
var replyTo: ActorRef = null
private def nodesForReadWrite(): Vector[UniqueAddress] = {
if (settings.preferOldest)
membersByAge.iterator.map(_.uniqueAddress).toVector
else
nodes.toVector
}
override protected[akka] def aroundReceive(rcv: Actor.Receive, msg: Any): Unit = {
replyTo = sender()
try {
@ -1547,7 +1624,16 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
} else {
context.actorOf(
ReadAggregator
.props(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo)
.props(
key,
consistency,
req,
selfUniqueAddress,
nodesForReadWrite(),
unreachable,
!settings.preferOldest,
localValue,
replyTo)
.withDispatcher(context.props.dispatcher))
}
}
@ -1632,6 +1718,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
case Some(d) => (newEnvelope.copy(data = d), None)
case None => (newEnvelope, None)
}
// When RequiresCausalDeliveryOfDeltas use deterministic order to so that sequence numbers
// of subsequent updates are in sync on the destination nodes.
// The order is also kept when prefer-oldest is enabled.
val shuffle = !(settings.preferOldest || writeDelta.exists(_.requiresCausalDeliveryOfDeltas))
val writeAggregator =
context.actorOf(
WriteAggregator
@ -1642,8 +1732,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
writeConsistency,
req,
selfUniqueAddress,
nodes,
nodesForReadWrite(),
unreachable,
shuffle,
replyTo,
durable)
.withDispatcher(context.props.dispatcher))
@ -1758,8 +1849,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency,
req,
selfUniqueAddress,
nodes,
nodesForReadWrite(),
unreachable,
!settings.preferOldest,
replyTo,
durable)
.withDispatcher(context.props.dispatcher))
@ -2106,6 +2198,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
nodes += m.uniqueAddress
weaklyUpNodes -= m.uniqueAddress
joiningNodes -= m.uniqueAddress
if (settings.preferOldest)
membersByAge += m
}
}
@ -2121,6 +2215,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
joiningNodes -= m.uniqueAddress
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
unreachable -= m.uniqueAddress
if (settings.preferOldest)
membersByAge -= m
deltaPropagationSelector.cleanupRemovedNode(m.uniqueAddress)
}
}
@ -2260,14 +2356,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
case object SendToSecondary
val MaxSecondaryNodes = 10
def calculateMajorityWithMinCap(minCap: Int, numberOfNodes: Int): Int = {
if (numberOfNodes <= minCap) {
numberOfNodes
} else {
val majority = numberOfNodes / 2 + 1
if (majority <= minCap) minCap
else majority
}
def calculateMajority(minCap: Int, numberOfNodes: Int, additional: Int): Int = {
val majority = numberOfNodes / 2 + 1
math.min(numberOfNodes, math.max(majority + additional, minCap))
}
}
@ -2278,30 +2369,33 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
import ReadWriteAggregator._
def timeout: FiniteDuration
def nodes: Set[UniqueAddress]
def nodes: Vector[UniqueAddress]
def unreachable: Set[UniqueAddress]
def reachableNodes: Set[UniqueAddress] = nodes.diff(unreachable)
def reachableNodes: Vector[UniqueAddress] = nodes.filterNot(unreachable)
def shuffle: Boolean
import context.dispatcher
var sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
var timeoutSchedule = context.system.scheduler.scheduleOnce(timeout, self, ReceiveTimeout)
var remaining = nodes.map(_.address)
var remaining = nodes.iterator.map(_.address).toSet
def doneWhenRemainingSize: Int
def primaryAndSecondaryNodes(
requiresCausalDeliveryOfDeltas: Boolean): (Vector[UniqueAddress], Vector[UniqueAddress]) = {
def primaryAndSecondaryNodes(): (Vector[UniqueAddress], Vector[UniqueAddress]) = {
val primarySize = nodes.size - doneWhenRemainingSize
if (primarySize >= nodes.size)
(nodes.toVector, Vector.empty[UniqueAddress])
(nodes, Vector.empty[UniqueAddress])
else {
// Prefer to use reachable nodes over the unreachable nodes first.
// When RequiresCausalDeliveryOfDeltas use deterministic order to so that sequence numbers of subsequent
// updates are in sync on the destination nodes.
// When RequiresCausalDeliveryOfDeltas (shuffle=false) use deterministic order to so that sequence numbers
// of subsequent updates are in sync on the destination nodes.
// The order is also kept when prefer-oldest is enabled.
val orderedNodes =
if (requiresCausalDeliveryOfDeltas) reachableNodes.toVector.sorted ++ unreachable.toVector.sorted
else scala.util.Random.shuffle(reachableNodes.toVector) ++ scala.util.Random.shuffle(unreachable.toVector)
if (shuffle)
scala.util.Random.shuffle(reachableNodes) ++ scala.util.Random.shuffle(unreachable.toVector)
else
reachableNodes ++ unreachable
val (p, s) = orderedNodes.splitAt(primarySize)
(p, s.take(MaxSecondaryNodes))
}
@ -2328,8 +2422,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.WriteConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
shuffle: Boolean,
replyTo: ActorRef,
durable: Boolean): Props =
Props(
@ -2342,6 +2437,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
selfUniqueAddress,
nodes,
unreachable,
shuffle,
replyTo,
durable)).withDeploy(Deploy.local)
}
@ -2356,8 +2452,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.WriteConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
override val nodes: Set[UniqueAddress],
override val nodes: Vector[UniqueAddress],
override val unreachable: Set[UniqueAddress],
override val shuffle: Boolean,
replyTo: ActorRef,
durable: Boolean)
extends ReadWriteAggregator
@ -2370,11 +2467,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
override def timeout: FiniteDuration = consistency.timeout
override val doneWhenRemainingSize = consistency match {
case WriteTo(n, _) => nodes.size - (n - 1)
case _: WriteAll => 0
case WriteTo(n, _) => nodes.size - (n - 1)
case _: WriteAll => 0
case WriteMajority(_, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val w = calculateMajorityWithMinCap(minCap, N)
val w = calculateMajority(minCap, N, 0)
log.debug("WriteMajority [{}] [{}] of [{}].", key, w, N)
N - w
case WriteMajorityPlus(_, additional, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val w = calculateMajority(minCap, N, additional)
log.debug("WriteMajorityPlus [{}] [{}] of [{}].", key, w, N)
N - w
case WriteLocal =>
throw new IllegalArgumentException("WriteLocal not supported by WriteAggregator")
@ -2389,13 +2494,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
var gotLocalStoreReply = !durable
var gotWriteNackFrom = Set.empty[Address]
private val (primaryNodes, secondaryNodes) = {
val requiresCausalDeliveryOfDeltas = delta match {
case None => false
case Some(d) => d.dataEnvelope.data.isInstanceOf[RequiresCausalDeliveryOfDeltas]
}
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas)
}
private val (primaryNodes, secondaryNodes) = primaryAndSecondaryNodes()
override def preStart(): Unit = {
val msg = deltaMsg match {
@ -2479,11 +2578,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.ReadConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
shuffle: Boolean,
localValue: Option[Replicator.Internal.DataEnvelope],
replyTo: ActorRef): Props =
Props(new ReadAggregator(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo))
Props(
new ReadAggregator(key, consistency, req, selfUniqueAddress, nodes, unreachable, shuffle, localValue, replyTo))
.withDeploy(Deploy.local)
}
@ -2496,11 +2597,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.ReadConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
override val nodes: Set[UniqueAddress],
override val nodes: Vector[UniqueAddress],
override val unreachable: Set[UniqueAddress],
override val shuffle: Boolean,
localValue: Option[Replicator.Internal.DataEnvelope],
replyTo: ActorRef)
extends ReadWriteAggregator {
extends ReadWriteAggregator
with ActorLogging {
import Replicator._
import Replicator.Internal._
@ -2510,11 +2613,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
var result = localValue
override val doneWhenRemainingSize = consistency match {
case ReadFrom(n, _) => nodes.size - (n - 1)
case _: ReadAll => 0
case ReadFrom(n, _) => nodes.size - (n - 1)
case _: ReadAll => 0
case ReadMajority(_, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val r = calculateMajorityWithMinCap(minCap, N)
val r = calculateMajority(minCap, N, 0)
log.debug("ReadMajority [{}] [{}] of [{}].", key, r, N)
N - r
case ReadMajorityPlus(_, additional, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val r = calculateMajority(minCap, N, additional)
log.debug("ReadMajorityPlus [{}] [{}] of [{}].", key, r, N)
N - r
case ReadLocal =>
throw new IllegalArgumentException("ReadLocal not supported by ReadAggregator")
@ -2522,9 +2633,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val readMsg = Read(key.id, Some(selfUniqueAddress))
private val (primaryNodes, secondaryNodes) = {
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas = false)
}
private val (primaryNodes, secondaryNodes) = primaryAndSecondaryNodes()
override def preStart(): Unit = {
primaryNodes.foreach { replica(_) ! readMsg }

View file

@ -338,21 +338,27 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
}
private def getToProto(get: Get[_]): dm.Get = {
val consistencyValue = get.consistency match {
case ReadLocal => 1
case ReadFrom(n, _) => n
case _: ReadMajority => 0
case _: ReadAll => -1
}
val timoutInMillis = get.consistency.timeout.toMillis
require(timoutInMillis <= 0XFFFFFFFFL, "Timeouts must fit in a 32-bit unsigned int")
val b = dm.Get
.newBuilder()
.setKey(otherMessageToProto(get.key))
.setConsistency(consistencyValue)
.setTimeout(timoutInMillis.toInt)
val b = dm.Get.newBuilder().setKey(otherMessageToProto(get.key)).setTimeout(timoutInMillis.toInt)
get.consistency match {
case ReadLocal => b.setConsistency(1)
case ReadFrom(n, _) => b.setConsistency(n)
case ReadMajority(_, minCap) =>
b.setConsistency(0)
if (minCap != 0)
b.setConsistencyMinCap(minCap)
case ReadMajorityPlus(_, additional, minCap) =>
b.setConsistency(0)
if (minCap != 0)
b.setConsistencyMinCap(minCap)
if (additional != 0)
b.setConsistencyAdditional(additional)
case _: ReadAll =>
b.setConsistency(-1)
}
get.request.foreach(o => b.setRequest(otherMessageToProto(o)))
b.build()
@ -367,8 +373,13 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
val timeout =
if (get.getTimeout < 0) Duration(Int.MaxValue.toLong + (get.getTimeout - Int.MaxValue), TimeUnit.MILLISECONDS)
else Duration(get.getTimeout.toLong, TimeUnit.MILLISECONDS)
def minCap = if (get.hasConsistencyMinCap) get.getConsistencyMinCap else 0
val consistency = get.getConsistency match {
case 0 => ReadMajority(timeout)
case 0 =>
if (get.hasConsistencyAdditional)
ReadMajorityPlus(timeout, get.getConsistencyAdditional, minCap)
else
ReadMajority(timeout, minCap)
case -1 => ReadAll(timeout)
case 1 => ReadLocal
case n => ReadFrom(n, timeout)