* 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:
parent
d415211bf8
commit
4ba835d328
17 changed files with 767 additions and 196 deletions
|
|
@ -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 }
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue