Avoid false removals in ClusterReceptionist, #26284
* The scenario was (probably) that a node was restarted with same host:port and then didn't join the same cluster. The DData Replicator in the original cluster would continue sending messages to the new incarnation resulting in false removals. * The fix is that DData Replicator includes the system uid of the sending or target system in messages and if recipient gets a message that is from/to unknown it will discard it and thereby not spreading information across different clusters. * Reproduced in ClusterReceptionistSpec * Much hardening of other things in ClusterReceptionistSpec * There are also some improvements to ClusterReceptionist to not leak Listing with refs of removed nodes. * use ClusterShuttingDown * The reason for using sender system uid instead of target uid in messages like Read and Write is that then the optimization for sending same message to many destinations can remain.
This commit is contained in:
parent
3cbda93496
commit
825d90bf63
16 changed files with 1714 additions and 396 deletions
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
// Generated by the protocol buffer compiler. DO NOT EDIT!
|
||||
|
|
|
|||
File diff suppressed because it is too large
Load diff
|
|
@ -2,3 +2,18 @@
|
|||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.ORSet.clear")
|
||||
ProblemFilters.exclude[DirectAbstractMethodProblem]("akka.cluster.ddata.PruningState.addSeen")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.PruningState.addSeen")
|
||||
|
||||
# #26284 adding uid to internal messages
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.Replicator#Internal*")
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.Replicator$Internal*")
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.ReadAggregator*")
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.WriteAggregator*")
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.ReadWriteAggregator*")
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.protobuf.msg.ReplicatorMessages*")
|
||||
ProblemFilters.exclude[Problem]("akka.cluster.ddata.DeltaPropagationSelector*")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator.receiveGossip")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator.receiveStatus")
|
||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.Replicator.gossipTo")
|
||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.Replicator.replica")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator.receiveWeaklyUpMemberUp")
|
||||
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ message Get {
|
|||
required OtherMessage key = 1;
|
||||
required sint32 consistency = 2;
|
||||
required uint32 timeout = 3;
|
||||
optional OtherMessage request = 4;
|
||||
optional OtherMessage request = 4;
|
||||
}
|
||||
|
||||
message GetSuccess {
|
||||
|
|
@ -48,6 +48,7 @@ message Changed {
|
|||
message Write {
|
||||
required string key = 1;
|
||||
required DataEnvelope envelope = 2;
|
||||
optional UniqueAddress fromNode = 3;
|
||||
}
|
||||
|
||||
// message WriteAck, via Empty
|
||||
|
|
@ -57,6 +58,7 @@ message Empty {
|
|||
|
||||
message Read {
|
||||
required string key = 1;
|
||||
optional UniqueAddress fromNode = 2;
|
||||
}
|
||||
|
||||
message ReadResult {
|
||||
|
|
@ -86,6 +88,8 @@ message Status {
|
|||
required uint32 chunk = 1;
|
||||
required uint32 totChunks = 2;
|
||||
repeated Entry entries = 3;
|
||||
optional sfixed64 toSystemUid = 4;
|
||||
optional sfixed64 fromSystemUid = 5;
|
||||
}
|
||||
|
||||
message Gossip {
|
||||
|
|
@ -96,6 +100,8 @@ message Gossip {
|
|||
|
||||
required bool sendBack = 1;
|
||||
repeated Entry entries = 2;
|
||||
optional sfixed64 toSystemUid = 3;
|
||||
optional sfixed64 fromSystemUid = 4;
|
||||
}
|
||||
|
||||
message DeltaPropagation {
|
||||
|
|
@ -108,7 +114,7 @@ message DeltaPropagation {
|
|||
|
||||
required UniqueAddress fromNode = 1;
|
||||
repeated Entry entries = 2;
|
||||
optional bool reply = 3; // no reply if not set
|
||||
optional bool reply = 3; // no reply if not set
|
||||
}
|
||||
|
||||
message UniqueAddress {
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package akka.cluster.ddata
|
|||
|
||||
import scala.collection.immutable.TreeMap
|
||||
|
||||
import akka.actor.Address
|
||||
import akka.annotation.InternalApi
|
||||
import akka.cluster.UniqueAddress
|
||||
import akka.cluster.ddata.Key.KeyId
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
||||
|
|
@ -25,12 +25,12 @@ private[akka] trait DeltaPropagationSelector {
|
|||
def propagationCount: Long = _propagationCount
|
||||
private var deltaCounter = Map.empty[KeyId, Long]
|
||||
private var deltaEntries = Map.empty[KeyId, TreeMap[Long, ReplicatedData]]
|
||||
private var deltaSentToNode = Map.empty[KeyId, Map[Address, Long]]
|
||||
private var deltaSentToNode = Map.empty[KeyId, Map[UniqueAddress, Long]]
|
||||
private var deltaNodeRoundRobinCounter = 0L
|
||||
|
||||
def gossipIntervalDivisor: Int
|
||||
|
||||
def allNodes: Vector[Address]
|
||||
def allNodes: Vector[UniqueAddress]
|
||||
|
||||
def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation
|
||||
|
||||
|
|
@ -68,7 +68,7 @@ private[akka] trait DeltaPropagationSelector {
|
|||
math.min(math.max((allNodesSize / gossipIntervalDivisor) + 1, 2), math.min(allNodesSize, 10))
|
||||
}
|
||||
|
||||
def collectPropagations(): Map[Address, DeltaPropagation] = {
|
||||
def collectPropagations(): Map[UniqueAddress, DeltaPropagation] = {
|
||||
_propagationCount += 1
|
||||
val all = allNodes
|
||||
if (all.isEmpty)
|
||||
|
|
@ -90,7 +90,7 @@ private[akka] trait DeltaPropagationSelector {
|
|||
}
|
||||
deltaNodeRoundRobinCounter += sliceSize
|
||||
|
||||
var result = Map.empty[Address, DeltaPropagation]
|
||||
var result = Map.empty[UniqueAddress, DeltaPropagation]
|
||||
|
||||
var cache = Map.empty[(KeyId, Long, Long), ReplicatedData]
|
||||
slice.foreach { node =>
|
||||
|
|
@ -99,7 +99,7 @@ private[akka] trait DeltaPropagationSelector {
|
|||
var deltas = Map.empty[KeyId, (ReplicatedData, Long, Long)]
|
||||
deltaEntries.foreach {
|
||||
case (key, entries) =>
|
||||
val deltaSentToNodeForKey = deltaSentToNode.getOrElse(key, TreeMap.empty[Address, Long])
|
||||
val deltaSentToNodeForKey = deltaSentToNode.getOrElse(key, TreeMap.empty[UniqueAddress, Long])
|
||||
val j = deltaSentToNodeForKey.getOrElse(node, 0L)
|
||||
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
|
||||
if (deltaEntriesAfterJ.nonEmpty) {
|
||||
|
|
@ -160,7 +160,7 @@ private[akka] trait DeltaPropagationSelector {
|
|||
}
|
||||
}
|
||||
|
||||
private def findSmallestVersionPropagatedToAllNodes(key: KeyId, all: Vector[Address]): Long = {
|
||||
private def findSmallestVersionPropagatedToAllNodes(key: KeyId, all: Vector[UniqueAddress]): Long = {
|
||||
deltaSentToNode.get(key) match {
|
||||
case None => 0L
|
||||
case Some(deltaSentToNodeForKey) =>
|
||||
|
|
@ -188,7 +188,7 @@ private[akka] trait DeltaPropagationSelector {
|
|||
}
|
||||
}
|
||||
|
||||
def cleanupRemovedNode(address: Address): Unit = {
|
||||
def cleanupRemovedNode(address: UniqueAddress): Unit = {
|
||||
deltaSentToNode = deltaSentToNode.map {
|
||||
case (key, deltaSentToNodeForKey) =>
|
||||
key -> (deltaSentToNodeForKey - address)
|
||||
|
|
|
|||
|
|
@ -5,15 +5,18 @@
|
|||
package akka.cluster.ddata
|
||||
|
||||
import java.security.MessageDigest
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.mutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
|
||||
import scala.util.Failure
|
||||
import scala.util.Success
|
||||
import scala.util.Try
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorLogging
|
||||
import akka.actor.ActorRef
|
||||
|
|
@ -34,24 +37,31 @@ import akka.serialization.SerializationExtension
|
|||
import akka.util.ByteString
|
||||
import com.typesafe.config.Config
|
||||
import java.util.function.{ Function => JFunction }
|
||||
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.actor.DeadLetterSuppression
|
||||
import akka.cluster.ddata.Key.KeyR
|
||||
import java.util.Optional
|
||||
|
||||
import akka.cluster.ddata.DurableStore._
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.actor.ActorInitializationException
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
import akka.util.Helpers.toRootLowerCase
|
||||
import akka.actor.Cancellable
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
import akka.cluster.ddata.Key.KeyId
|
||||
import akka.annotation.InternalApi
|
||||
import scala.collection.immutable.TreeSet
|
||||
|
||||
import akka.cluster.MemberStatus
|
||||
import scala.annotation.varargs
|
||||
|
||||
import akka.event.Logging
|
||||
import akka.util.JavaDurationConverters._
|
||||
import akka.util.ccompat._
|
||||
|
||||
|
|
@ -790,10 +800,20 @@ object Replicator {
|
|||
case object DeltaPropagationTick
|
||||
case object RemovedNodePruningTick
|
||||
case object ClockTick
|
||||
final case class Write(key: KeyId, envelope: DataEnvelope) extends ReplicatorMessage
|
||||
sealed trait SendingSystemUid {
|
||||
// FIXME #26566: we can change from Option to UniqueAddress after supporting mixed rolling updates for some versions
|
||||
def fromNode: Option[UniqueAddress]
|
||||
}
|
||||
sealed trait DestinationSystemUid {
|
||||
// FIXME #26566: we can change from Option to Long after supporting mixed rolling updates for some versions
|
||||
def toSystemUid: Option[Long]
|
||||
}
|
||||
final case class Write(key: KeyId, envelope: DataEnvelope, fromNode: Option[UniqueAddress])
|
||||
extends ReplicatorMessage
|
||||
with SendingSystemUid
|
||||
case object WriteAck extends ReplicatorMessage with DeadLetterSuppression
|
||||
case object WriteNack extends ReplicatorMessage with DeadLetterSuppression
|
||||
final case class Read(key: KeyId) extends ReplicatorMessage
|
||||
final case class Read(key: KeyId, fromNode: Option[UniqueAddress]) extends ReplicatorMessage with SendingSystemUid
|
||||
final case class ReadResult(envelope: Option[DataEnvelope]) extends ReplicatorMessage with DeadLetterSuppression
|
||||
final case class ReadRepair(key: KeyId, envelope: DataEnvelope)
|
||||
case object ReadRepairAck
|
||||
|
|
@ -946,7 +966,14 @@ object Replicator {
|
|||
override def merge(that: ReplicatedData): ReplicatedData = DeletedData
|
||||
}
|
||||
|
||||
final case class Status(digests: Map[KeyId, Digest], chunk: Int, totChunks: Int) extends ReplicatorMessage {
|
||||
final case class Status(
|
||||
digests: Map[KeyId, Digest],
|
||||
chunk: Int,
|
||||
totChunks: Int,
|
||||
toSystemUid: Option[Long],
|
||||
fromSystemUid: Option[Long])
|
||||
extends ReplicatorMessage
|
||||
with DestinationSystemUid {
|
||||
override def toString: String =
|
||||
(digests
|
||||
.map {
|
||||
|
|
@ -954,11 +981,22 @@ object Replicator {
|
|||
})
|
||||
.mkString("Status(", ", ", ")")
|
||||
}
|
||||
final case class Gossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean) extends ReplicatorMessage
|
||||
final case class Gossip(
|
||||
updatedData: Map[KeyId, DataEnvelope],
|
||||
sendBack: Boolean,
|
||||
toSystemUid: Option[Long],
|
||||
fromSystemUid: Option[Long])
|
||||
extends ReplicatorMessage
|
||||
with DestinationSystemUid
|
||||
|
||||
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long)
|
||||
final case class DeltaPropagation(fromNode: UniqueAddress, reply: Boolean, deltas: Map[KeyId, Delta])
|
||||
final case class DeltaPropagation(_fromNode: UniqueAddress, reply: Boolean, deltas: Map[KeyId, Delta])
|
||||
extends ReplicatorMessage
|
||||
with SendingSystemUid {
|
||||
// FIXME we can change from Option to UniqueAddress after supporting mixed rolling updates for some versions,
|
||||
// i.e. remove this and rename _fromNode
|
||||
override def fromNode: Option[UniqueAddress] = Some(_fromNode)
|
||||
}
|
||||
object DeltaPropagation {
|
||||
|
||||
/**
|
||||
|
|
@ -1186,6 +1224,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
val cluster = Cluster(context.system)
|
||||
val selfAddress = cluster.selfAddress
|
||||
val selfUniqueAddress = cluster.selfUniqueAddress
|
||||
val selfFromSystemUid = Some(selfUniqueAddress.longUid)
|
||||
|
||||
require(!cluster.isTerminated, "Cluster node must not be terminated")
|
||||
require(
|
||||
|
|
@ -1223,9 +1262,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
val deltaPropagationSelector = new DeltaPropagationSelector {
|
||||
override val gossipIntervalDivisor = 5
|
||||
override def allNodes: Vector[Address] = {
|
||||
override def allNodes: Vector[UniqueAddress] = {
|
||||
// TODO optimize, by maintaining a sorted instance variable instead
|
||||
nodes.union(weaklyUpNodes).diff(unreachable).toVector.sorted
|
||||
Replicator.this.allNodes.diff(unreachable).toVector.sorted
|
||||
}
|
||||
|
||||
override def maxDeltaSize: Int = settings.maxDeltaSize
|
||||
|
|
@ -1256,11 +1295,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
.schedule(deltaPropagationInterval, deltaPropagationInterval, self, DeltaPropagationTick))
|
||||
} else None
|
||||
|
||||
// cluster nodes, doesn't contain selfAddress
|
||||
var nodes: Set[Address] = Set.empty
|
||||
// cluster nodes, doesn't contain selfAddress, doesn't contain joining and weaklyUp
|
||||
var nodes: Set[UniqueAddress] = Set.empty
|
||||
|
||||
// cluster weaklyUp nodes, doesn't contain selfAddress
|
||||
var weaklyUpNodes: Set[Address] = Set.empty
|
||||
var weaklyUpNodes: Set[UniqueAddress] = Set.empty
|
||||
|
||||
// cluster joining nodes, doesn't contain selfAddress
|
||||
var joiningNodes: Set[UniqueAddress] = Set.empty
|
||||
|
||||
// up and weaklyUp nodes, doesn't contain joining and not selfAddress
|
||||
private def allNodes: Set[UniqueAddress] = nodes.union(weaklyUpNodes)
|
||||
|
||||
private def isKnownNode(node: UniqueAddress): Boolean =
|
||||
nodes(node) || weaklyUpNodes(node) || joiningNodes(node) || selfUniqueAddress == node
|
||||
|
||||
var removedNodes: Map[UniqueAddress, Long] = Map.empty
|
||||
// all nodes sorted with the leader first
|
||||
|
|
@ -1271,7 +1319,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
// for pruning timeouts are based on clock that is only increased when all nodes are reachable
|
||||
var previousClockTime = System.nanoTime()
|
||||
var allReachableClockTime = 0L
|
||||
var unreachable = Set.empty[Address]
|
||||
var unreachable = Set.empty[UniqueAddress]
|
||||
|
||||
// the actual data
|
||||
var dataEntries = Map.empty[KeyId, (DataEnvelope, Digest)]
|
||||
|
|
@ -1397,32 +1445,66 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
// MUST use replyTo instead of sender() and forward from normalReceive, because of the stash in load
|
||||
val normalReceive: Receive = {
|
||||
case Get(key, consistency, req) => receiveGet(key, consistency, req)
|
||||
case u @ Update(key, writeC, req) => receiveUpdate(key, u.modify, writeC, req)
|
||||
case Read(key) => receiveRead(key)
|
||||
case Write(key, envelope) => receiveWrite(key, envelope)
|
||||
case ReadRepair(key, envelope) => receiveReadRepair(key, envelope)
|
||||
case DeltaPropagation(from, reply, deltas) => receiveDeltaPropagation(from, reply, deltas)
|
||||
case FlushChanges => receiveFlushChanges()
|
||||
case DeltaPropagationTick => receiveDeltaPropagationTick()
|
||||
case GossipTick => receiveGossipTick()
|
||||
case ClockTick => receiveClockTick()
|
||||
case Status(otherDigests, chunk, totChunks) => receiveStatus(otherDigests, chunk, totChunks)
|
||||
case Gossip(updatedData, sendBack) => receiveGossip(updatedData, sendBack)
|
||||
case Subscribe(key, subscriber) => receiveSubscribe(key, subscriber)
|
||||
case Unsubscribe(key, subscriber) => receiveUnsubscribe(key, subscriber)
|
||||
case Terminated(ref) => receiveTerminated(ref)
|
||||
case MemberWeaklyUp(m) => receiveWeaklyUpMemberUp(m)
|
||||
case MemberUp(m) => receiveMemberUp(m)
|
||||
case MemberRemoved(m, _) => receiveMemberRemoved(m)
|
||||
case evt: MemberEvent => receiveOtherMemberEvent(evt.member)
|
||||
case UnreachableMember(m) => receiveUnreachable(m)
|
||||
case ReachableMember(m) => receiveReachable(m)
|
||||
case GetKeyIds => receiveGetKeyIds()
|
||||
case Delete(key, consistency, req) => receiveDelete(key, consistency, req)
|
||||
case RemovedNodePruningTick => receiveRemovedNodePruningTick()
|
||||
case GetReplicaCount => receiveGetReplicaCount()
|
||||
case TestFullStateGossip(enabled) => fullStateGossipEnabled = enabled
|
||||
case msg: DestinationSystemUid =>
|
||||
msg.toSystemUid match {
|
||||
case Some(uid) if uid != selfUniqueAddress.longUid =>
|
||||
// When restarting a node with same host:port it is possible that a Replicator on another node
|
||||
// is sending messages to the restarted node even if it hasn't joined the same cluster.
|
||||
// Therefore we check that the message was intended for this incarnation and otherwise
|
||||
// it is discarded.
|
||||
log.info(
|
||||
"Ignoring message [{}] from [{}] intended for system uid [{}], self uid is [{}]",
|
||||
Logging.simpleName(msg),
|
||||
replyTo,
|
||||
uid,
|
||||
selfUniqueAddress.longUid)
|
||||
case _ =>
|
||||
msg match {
|
||||
case Status(otherDigests, chunk, totChunks, _, fromSystemUid) =>
|
||||
receiveStatus(otherDigests, chunk, totChunks, fromSystemUid)
|
||||
case Gossip(updatedData, sendBack, _, fromSystemUid) =>
|
||||
receiveGossip(updatedData, sendBack, fromSystemUid)
|
||||
}
|
||||
}
|
||||
|
||||
case msg: SendingSystemUid =>
|
||||
msg.fromNode match {
|
||||
case Some(fromNode) if !isKnownNode(fromNode) =>
|
||||
// When restarting a node with same host:port it is possible that a Replicator on another node
|
||||
// is sending messages to the restarted node even if it hasn't joined the same cluster.
|
||||
// Therefore we check that the message was from a known cluster member
|
||||
log.info("Ignoring message [{}] from [{}] unknown node [{}]", Logging.simpleName(msg), replyTo, fromNode)
|
||||
|
||||
case _ =>
|
||||
msg match {
|
||||
case Read(key, _) => receiveRead(key)
|
||||
case Write(key, envelope, _) => receiveWrite(key, envelope)
|
||||
case DeltaPropagation(from, reply, deltas) => receiveDeltaPropagation(from, reply, deltas)
|
||||
}
|
||||
}
|
||||
|
||||
case Get(key, consistency, req) => receiveGet(key, consistency, req)
|
||||
case u @ Update(key, writeC, req) => receiveUpdate(key, u.modify, writeC, req)
|
||||
case ReadRepair(key, envelope) => receiveReadRepair(key, envelope)
|
||||
case FlushChanges => receiveFlushChanges()
|
||||
case DeltaPropagationTick => receiveDeltaPropagationTick()
|
||||
case GossipTick => receiveGossipTick()
|
||||
case ClockTick => receiveClockTick()
|
||||
case Subscribe(key, subscriber) => receiveSubscribe(key, subscriber)
|
||||
case Unsubscribe(key, subscriber) => receiveUnsubscribe(key, subscriber)
|
||||
case Terminated(ref) => receiveTerminated(ref)
|
||||
case MemberJoined(m) => receiveMemberJoining(m)
|
||||
case MemberWeaklyUp(m) => receiveMemberWeaklyUp(m)
|
||||
case MemberUp(m) => receiveMemberUp(m)
|
||||
case MemberRemoved(m, _) => receiveMemberRemoved(m)
|
||||
case evt: MemberEvent => receiveOtherMemberEvent(evt.member)
|
||||
case UnreachableMember(m) => receiveUnreachable(m)
|
||||
case ReachableMember(m) => receiveReachable(m)
|
||||
case GetKeyIds => receiveGetKeyIds()
|
||||
case Delete(key, consistency, req) => receiveDelete(key, consistency, req)
|
||||
case RemovedNodePruningTick => receiveRemovedNodePruningTick()
|
||||
case GetReplicaCount => receiveGetReplicaCount()
|
||||
case TestFullStateGossip(enabled) => fullStateGossipEnabled = enabled
|
||||
}
|
||||
|
||||
def receiveGet(key: KeyR, consistency: ReadConsistency, req: Option[Any]): Unit = {
|
||||
|
|
@ -1438,7 +1520,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
} else
|
||||
context.actorOf(
|
||||
ReadAggregator
|
||||
.props(key, consistency, req, nodes, unreachable, localValue, replyTo)
|
||||
.props(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
}
|
||||
|
||||
|
|
@ -1521,7 +1603,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
val writeAggregator =
|
||||
context.actorOf(
|
||||
WriteAggregator
|
||||
.props(key, writeEnvelope, writeDelta, writeConsistency, req, nodes, unreachable, replyTo, durable)
|
||||
.props(
|
||||
key,
|
||||
writeEnvelope,
|
||||
writeDelta,
|
||||
writeConsistency,
|
||||
req,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
unreachable,
|
||||
replyTo,
|
||||
durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(
|
||||
|
|
@ -1630,7 +1722,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
val writeAggregator =
|
||||
context.actorOf(
|
||||
WriteAggregator
|
||||
.props(key, DeletedEnvelope, None, consistency, req, nodes, unreachable, replyTo, durable)
|
||||
.props(
|
||||
key,
|
||||
DeletedEnvelope,
|
||||
None,
|
||||
consistency,
|
||||
req,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
unreachable,
|
||||
replyTo,
|
||||
durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(
|
||||
|
|
@ -1811,13 +1913,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
def receiveGossipTick(): Unit = {
|
||||
if (fullStateGossipEnabled)
|
||||
selectRandomNode(nodes.union(weaklyUpNodes).toVector).foreach(gossipTo)
|
||||
selectRandomNode(allNodes.toVector).foreach(gossipTo)
|
||||
}
|
||||
|
||||
def gossipTo(address: Address): Unit = {
|
||||
def gossipTo(address: UniqueAddress): Unit = {
|
||||
val to = replica(address)
|
||||
val toSystemUid = Some(address.longUid)
|
||||
if (dataEntries.size <= maxDeltaElements) {
|
||||
val status = Status(dataEntries.map { case (key, (_, _)) => (key, getDigest(key)) }, chunk = 0, totChunks = 1)
|
||||
val status = Status(
|
||||
dataEntries.map { case (key, (_, _)) => (key, getDigest(key)) },
|
||||
chunk = 0,
|
||||
totChunks = 1,
|
||||
toSystemUid,
|
||||
selfFromSystemUid)
|
||||
to ! status
|
||||
} else {
|
||||
val totChunks = dataEntries.size / maxDeltaElements
|
||||
|
|
@ -1831,19 +1939,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
val chunk = (statusCount % totChunks).toInt
|
||||
val status = Status(dataEntries.collect {
|
||||
case (key, (_, _)) if math.abs(key.hashCode % totChunks) == chunk => (key, getDigest(key))
|
||||
}, chunk, totChunks)
|
||||
}, chunk, totChunks, toSystemUid, selfFromSystemUid)
|
||||
to ! status
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def selectRandomNode(addresses: immutable.IndexedSeq[Address]): Option[Address] =
|
||||
def selectRandomNode(addresses: immutable.IndexedSeq[UniqueAddress]): Option[UniqueAddress] =
|
||||
if (addresses.isEmpty) None else Some(addresses(ThreadLocalRandom.current.nextInt(addresses.size)))
|
||||
|
||||
def replica(address: Address): ActorSelection =
|
||||
context.actorSelection(self.path.toStringWithAddress(address))
|
||||
def replica(node: UniqueAddress): ActorSelection =
|
||||
context.actorSelection(self.path.toStringWithAddress(node.address))
|
||||
|
||||
def receiveStatus(otherDigests: Map[KeyId, Digest], chunk: Int, totChunks: Int): Unit = {
|
||||
def receiveStatus(otherDigests: Map[KeyId, Digest], chunk: Int, totChunks: Int, fromSystemUid: Option[Long]): Unit = {
|
||||
if (log.isDebugEnabled)
|
||||
log.debug(
|
||||
"Received gossip status from [{}], chunk [{}] of [{}] containing [{}]",
|
||||
|
|
@ -1868,7 +1976,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
if (keys.nonEmpty) {
|
||||
if (log.isDebugEnabled)
|
||||
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
|
||||
val g = Gossip(keys.iterator.map(k => k -> getData(k).get).toMap, sendBack = otherDifferentKeys.nonEmpty)
|
||||
val g = Gossip(
|
||||
keys.iterator.map(k => k -> getData(k).get).toMap,
|
||||
sendBack = otherDifferentKeys.nonEmpty,
|
||||
fromSystemUid,
|
||||
selfFromSystemUid)
|
||||
replyTo ! g
|
||||
}
|
||||
val myMissingKeys = otherKeys.diff(myKeys)
|
||||
|
|
@ -1878,12 +1990,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
"Sending gossip status to [{}], requesting missing [{}]",
|
||||
replyTo.path.address,
|
||||
myMissingKeys.mkString(", "))
|
||||
val status = Status(myMissingKeys.iterator.map(k => k -> NotFoundDigest).toMap, chunk, totChunks)
|
||||
val status = Status(
|
||||
myMissingKeys.iterator.map(k => k -> NotFoundDigest).toMap,
|
||||
chunk,
|
||||
totChunks,
|
||||
fromSystemUid,
|
||||
selfFromSystemUid)
|
||||
replyTo ! status
|
||||
}
|
||||
}
|
||||
|
||||
def receiveGossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean): Unit = {
|
||||
def receiveGossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean, fromSystemUid: Option[Long]): Unit = {
|
||||
if (log.isDebugEnabled)
|
||||
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
|
||||
var replyData = Map.empty[KeyId, DataEnvelope]
|
||||
|
|
@ -1899,7 +2016,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
}
|
||||
if (sendBack && replyData.nonEmpty)
|
||||
replyTo ! Gossip(replyData, sendBack = false)
|
||||
replyTo ! Gossip(replyData, sendBack = false, fromSystemUid, selfFromSystemUid)
|
||||
}
|
||||
|
||||
def receiveSubscribe(key: KeyR, subscriber: ActorRef): Unit = {
|
||||
|
|
@ -1943,16 +2060,23 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
}
|
||||
|
||||
def receiveWeaklyUpMemberUp(m: Member): Unit =
|
||||
def receiveMemberJoining(m: Member): Unit =
|
||||
if (matchingRole(m) && m.address != selfAddress)
|
||||
weaklyUpNodes += m.address
|
||||
joiningNodes += m.uniqueAddress
|
||||
|
||||
def receiveMemberWeaklyUp(m: Member): Unit =
|
||||
if (matchingRole(m) && m.address != selfAddress) {
|
||||
weaklyUpNodes += m.uniqueAddress
|
||||
joiningNodes -= m.uniqueAddress
|
||||
}
|
||||
|
||||
def receiveMemberUp(m: Member): Unit =
|
||||
if (matchingRole(m)) {
|
||||
leader += m
|
||||
if (m.address != selfAddress) {
|
||||
nodes += m.address
|
||||
weaklyUpNodes -= m.address
|
||||
nodes += m.uniqueAddress
|
||||
weaklyUpNodes -= m.uniqueAddress
|
||||
joiningNodes -= m.uniqueAddress
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1960,14 +2084,15 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
if (m.address == selfAddress)
|
||||
context.stop(self)
|
||||
else if (matchingRole(m)) {
|
||||
log.debug("adding removed node [{}] from MemberRemoved", m.uniqueAddress)
|
||||
// filter, it's possible that the ordering is changed since it based on MemberStatus
|
||||
leader = leader.filterNot(_.uniqueAddress == m.uniqueAddress)
|
||||
nodes -= m.address
|
||||
weaklyUpNodes -= m.address
|
||||
log.debug("adding removed node [{}] from MemberRemoved", m.uniqueAddress)
|
||||
nodes -= m.uniqueAddress
|
||||
weaklyUpNodes -= m.uniqueAddress
|
||||
joiningNodes -= m.uniqueAddress
|
||||
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
|
||||
unreachable -= m.address
|
||||
deltaPropagationSelector.cleanupRemovedNode(m.address)
|
||||
unreachable -= m.uniqueAddress
|
||||
deltaPropagationSelector.cleanupRemovedNode(m.uniqueAddress)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1979,10 +2104,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
def receiveUnreachable(m: Member): Unit =
|
||||
if (matchingRole(m)) unreachable += m.address
|
||||
if (matchingRole(m)) unreachable += m.uniqueAddress
|
||||
|
||||
def receiveReachable(m: Member): Unit =
|
||||
if (matchingRole(m)) unreachable -= m.address
|
||||
if (matchingRole(m)) unreachable -= m.uniqueAddress
|
||||
|
||||
def receiveClockTick(): Unit = {
|
||||
val now = System.nanoTime()
|
||||
|
|
@ -2004,11 +2129,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
def collectRemovedNodes(): Unit = {
|
||||
val knownNodes = nodes.union(weaklyUpNodes).union(removedNodes.keySet.map(_.address))
|
||||
val knownNodes = allNodes.union(removedNodes.keySet)
|
||||
val newRemovedNodes =
|
||||
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
|
||||
case (acc, (_, (DataEnvelope(data: RemovedNodePruning, _, _), _))) =>
|
||||
acc.union(data.modifiedByNodes.filterNot(n => n == selfUniqueAddress || knownNodes(n.address)))
|
||||
acc.union(data.modifiedByNodes.filterNot(n => n == selfUniqueAddress || knownNodes(n)))
|
||||
case (acc, _) =>
|
||||
acc
|
||||
}
|
||||
|
|
@ -2023,7 +2148,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
// initiate pruning for removed nodes
|
||||
val removedSet: Set[UniqueAddress] = removedNodes.iterator
|
||||
.collect {
|
||||
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) => r
|
||||
case (r, t) if (allReachableClockTime - t) > maxPruningDisseminationNanos => r
|
||||
}
|
||||
.to(immutable.Set)
|
||||
|
||||
|
|
@ -2053,7 +2178,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
def performRemovedNodePruning(): Unit = {
|
||||
// perform pruning when all seen Init
|
||||
val allNodes = nodes.union(weaklyUpNodes)
|
||||
val all = allNodes
|
||||
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
|
||||
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
|
||||
dataEntries.foreach {
|
||||
|
|
@ -2061,7 +2186,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
pruning.foreach {
|
||||
case (removed, PruningInitialized(owner, seen))
|
||||
if owner == selfUniqueAddress
|
||||
&& (allNodes.isEmpty || allNodes.forall(seen)) =>
|
||||
&& (all.isEmpty || all.forall(n => seen(n.address))) =>
|
||||
val newEnvelope = envelope.prune(removed, if (isDurable(key)) durablePruningPerformed else pruningPerformed)
|
||||
log.debug("Perform pruning of [{}] from [{}] to [{}]", key, removed, selfUniqueAddress)
|
||||
setData(key, newEnvelope)
|
||||
|
|
@ -2124,22 +2249,23 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
import ReadWriteAggregator._
|
||||
|
||||
def timeout: FiniteDuration
|
||||
def nodes: Set[Address]
|
||||
def unreachable: Set[Address]
|
||||
def reachableNodes: Set[Address] = nodes.diff(unreachable)
|
||||
def nodes: Set[UniqueAddress]
|
||||
def unreachable: Set[UniqueAddress]
|
||||
def reachableNodes: Set[UniqueAddress] = nodes.diff(unreachable)
|
||||
|
||||
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
|
||||
var remaining = nodes.map(_.address)
|
||||
|
||||
def doneWhenRemainingSize: Int
|
||||
|
||||
def primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas: Boolean): (Vector[Address], Vector[Address]) = {
|
||||
def primaryAndSecondaryNodes(
|
||||
requiresCausalDeliveryOfDeltas: Boolean): (Vector[UniqueAddress], Vector[UniqueAddress]) = {
|
||||
val primarySize = nodes.size - doneWhenRemainingSize
|
||||
if (primarySize >= nodes.size)
|
||||
(nodes.toVector, Vector.empty[Address])
|
||||
(nodes.toVector, 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
|
||||
|
|
@ -2157,8 +2283,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
timeoutSchedule.cancel()
|
||||
}
|
||||
|
||||
def replica(address: Address): ActorSelection =
|
||||
context.actorSelection(context.parent.path.toStringWithAddress(address))
|
||||
def replica(node: UniqueAddress): ActorSelection =
|
||||
context.actorSelection(context.parent.path.toStringWithAddress(node.address))
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -2172,12 +2298,23 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
delta: Option[Replicator.Internal.Delta],
|
||||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
nodes: Set[UniqueAddress],
|
||||
unreachable: Set[UniqueAddress],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean): Props =
|
||||
Props(new WriteAggregator(key, envelope, delta, consistency, req, nodes, unreachable, replyTo, durable))
|
||||
.withDeploy(Deploy.local)
|
||||
Props(
|
||||
new WriteAggregator(
|
||||
key,
|
||||
envelope,
|
||||
delta,
|
||||
consistency,
|
||||
req,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
unreachable,
|
||||
replyTo,
|
||||
durable)).withDeploy(Deploy.local)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -2189,18 +2326,18 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
delta: Option[Replicator.Internal.Delta],
|
||||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
override val unreachable: Set[Address],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
override val nodes: Set[UniqueAddress],
|
||||
override val unreachable: Set[UniqueAddress],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean)
|
||||
extends ReadWriteAggregator {
|
||||
extends ReadWriteAggregator
|
||||
with ActorLogging {
|
||||
|
||||
import Replicator._
|
||||
import Replicator.Internal._
|
||||
import ReadWriteAggregator._
|
||||
|
||||
val selfUniqueAddress = Cluster(context.system).selfUniqueAddress
|
||||
|
||||
override def timeout: FiniteDuration = consistency.timeout
|
||||
|
||||
override val doneWhenRemainingSize = consistency match {
|
||||
|
|
@ -2214,7 +2351,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
throw new IllegalArgumentException("WriteLocal not supported by WriteAggregator")
|
||||
}
|
||||
|
||||
val writeMsg = Write(key.id, envelope)
|
||||
val writeMsg = Write(key.id, envelope, Some(selfUniqueAddress))
|
||||
val deltaMsg = delta match {
|
||||
case None => None
|
||||
case Some(d) => Some(DeltaPropagation(selfUniqueAddress, reply = true, Map(key.id -> d)))
|
||||
|
|
@ -2267,7 +2404,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
// 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 }
|
||||
primaryNodes.foreach { to =>
|
||||
if (remaining(to.address))
|
||||
replica(to) ! writeMsg
|
||||
}
|
||||
}
|
||||
secondaryNodes.foreach { replica(_) ! writeMsg }
|
||||
case ReceiveTimeout =>
|
||||
|
|
@ -2309,11 +2449,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
key: KeyR,
|
||||
consistency: Replicator.ReadConsistency,
|
||||
req: Option[Any],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
nodes: Set[UniqueAddress],
|
||||
unreachable: Set[UniqueAddress],
|
||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||
replyTo: ActorRef): Props =
|
||||
Props(new ReadAggregator(key, consistency, req, nodes, unreachable, localValue, replyTo)).withDeploy(Deploy.local)
|
||||
Props(new ReadAggregator(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo))
|
||||
.withDeploy(Deploy.local)
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -2324,8 +2466,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
key: KeyR,
|
||||
consistency: Replicator.ReadConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
override val unreachable: Set[Address],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
override val nodes: Set[UniqueAddress],
|
||||
override val unreachable: Set[UniqueAddress],
|
||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||
replyTo: ActorRef)
|
||||
extends ReadWriteAggregator {
|
||||
|
|
@ -2348,7 +2491,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
throw new IllegalArgumentException("ReadLocal not supported by ReadAggregator")
|
||||
}
|
||||
|
||||
val readMsg = Read(key.id)
|
||||
val readMsg = Read(key.id, Some(selfUniqueAddress))
|
||||
|
||||
private val (primaryNodes, secondaryNodes) = {
|
||||
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas = false)
|
||||
|
|
|
|||
|
|
@ -50,14 +50,14 @@ private object ReplicatedDataSerializer {
|
|||
@silent
|
||||
private final def compareKeys(t1: Any, t2: Any): Int = (t1, t2) match {
|
||||
case (k1: String, k2: String) => k1.compareTo(k2)
|
||||
case (k1: String, k2) => -1
|
||||
case (k1, k2: String) => 1
|
||||
case (_: String, _) => -1
|
||||
case (_, _: String) => 1
|
||||
case (k1: Int, k2: Int) => k1.compareTo(k2)
|
||||
case (k1: Int, k2) => -1
|
||||
case (k1, k2: Int) => 1
|
||||
case (_: Int, _) => -1
|
||||
case (_, _: Int) => 1
|
||||
case (k1: Long, k2: Long) => k1.compareTo(k2)
|
||||
case (k1: Long, k2) => -1
|
||||
case (k1, k2: Long) => 1
|
||||
case (_: Long, _) => -1
|
||||
case (_, _: Long) => 1
|
||||
case (k1: OtherMessage, k2: OtherMessage) => OtherMessageComparator.compare(k1, k2)
|
||||
case (k1, k2) =>
|
||||
throw new IllegalStateException(
|
||||
|
|
|
|||
|
|
@ -267,15 +267,21 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
case (key, digest) =>
|
||||
b.addEntries(dm.Status.Entry.newBuilder().setKey(key).setDigest(ByteString.copyFrom(digest.toArray)))
|
||||
}
|
||||
b.setToSystemUid(status.toSystemUid.get)
|
||||
b.setFromSystemUid(status.fromSystemUid.get)
|
||||
b.build()
|
||||
}
|
||||
|
||||
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
||||
val status = dm.Status.parseFrom(bytes)
|
||||
val toSystemUid = if (status.hasToSystemUid) Some(status.getToSystemUid) else None
|
||||
val fromSystemUid = if (status.hasFromSystemUid) Some(status.getFromSystemUid) else None
|
||||
Status(
|
||||
status.getEntriesList.asScala.iterator.map(e => e.getKey -> AkkaByteString(e.getDigest.toByteArray())).toMap,
|
||||
status.getChunk,
|
||||
status.getTotChunks)
|
||||
status.getTotChunks,
|
||||
toSystemUid,
|
||||
fromSystemUid)
|
||||
}
|
||||
|
||||
private def gossipToProto(gossip: Gossip): dm.Gossip = {
|
||||
|
|
@ -284,18 +290,24 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
case (key, data) =>
|
||||
b.addEntries(dm.Gossip.Entry.newBuilder().setKey(key).setEnvelope(dataEnvelopeToProto(data)))
|
||||
}
|
||||
b.setToSystemUid(gossip.toSystemUid.get)
|
||||
b.setFromSystemUid(gossip.fromSystemUid.get)
|
||||
b.build()
|
||||
}
|
||||
|
||||
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
||||
val gossip = dm.Gossip.parseFrom(decompress(bytes))
|
||||
val toSystemUid = if (gossip.hasToSystemUid) Some(gossip.getToSystemUid) else None
|
||||
val fromSystemUid = if (gossip.hasFromSystemUid) Some(gossip.getFromSystemUid) else None
|
||||
Gossip(
|
||||
gossip.getEntriesList.asScala.iterator.map(e => e.getKey -> dataEnvelopeFromProto(e.getEnvelope)).toMap,
|
||||
sendBack = gossip.getSendBack)
|
||||
sendBack = gossip.getSendBack,
|
||||
toSystemUid,
|
||||
fromSystemUid)
|
||||
}
|
||||
|
||||
private def deltaPropagationToProto(deltaPropagation: DeltaPropagation): dm.DeltaPropagation = {
|
||||
val b = dm.DeltaPropagation.newBuilder().setFromNode(uniqueAddressToProto(deltaPropagation.fromNode))
|
||||
val b = dm.DeltaPropagation.newBuilder().setFromNode(uniqueAddressToProto(deltaPropagation._fromNode))
|
||||
if (deltaPropagation.reply)
|
||||
b.setReply(deltaPropagation.reply)
|
||||
deltaPropagation.deltas.foreach {
|
||||
|
|
@ -513,18 +525,27 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
}
|
||||
|
||||
private def writeToProto(write: Write): dm.Write =
|
||||
dm.Write.newBuilder().setKey(write.key).setEnvelope(dataEnvelopeToProto(write.envelope)).build()
|
||||
dm.Write
|
||||
.newBuilder()
|
||||
.setKey(write.key)
|
||||
.setEnvelope(dataEnvelopeToProto(write.envelope))
|
||||
.setFromNode(uniqueAddressToProto(write.fromNode.get))
|
||||
.build()
|
||||
|
||||
private def writeFromBinary(bytes: Array[Byte]): Write = {
|
||||
val write = dm.Write.parseFrom(bytes)
|
||||
Write(write.getKey, dataEnvelopeFromProto(write.getEnvelope))
|
||||
val fromNode = if (write.hasFromNode) Some(uniqueAddressFromProto(write.getFromNode)) else None
|
||||
Write(write.getKey, dataEnvelopeFromProto(write.getEnvelope), fromNode)
|
||||
}
|
||||
|
||||
private def readToProto(read: Read): dm.Read =
|
||||
dm.Read.newBuilder().setKey(read.key).build()
|
||||
dm.Read.newBuilder().setKey(read.key).setFromNode(uniqueAddressToProto(read.fromNode.get)).build()
|
||||
|
||||
private def readFromBinary(bytes: Array[Byte]): Read =
|
||||
Read(dm.Read.parseFrom(bytes).getKey)
|
||||
private def readFromBinary(bytes: Array[Byte]): Read = {
|
||||
val read = dm.Read.parseFrom(bytes)
|
||||
val fromNode = if (read.hasFromNode) Some(uniqueAddressFromProto(read.getFromNode)) else None
|
||||
Read(read.getKey, fromNode)
|
||||
}
|
||||
|
||||
private def readResultToProto(readResult: ReadResult): dm.ReadResult = {
|
||||
val b = dm.ReadResult.newBuilder()
|
||||
|
|
|
|||
|
|
@ -181,6 +181,12 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
|||
values should ===(Set(10))
|
||||
}
|
||||
|
||||
// all must at least have seen it as joining
|
||||
awaitAssert({
|
||||
cluster3.state.members.size should ===(4)
|
||||
cluster3.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
}, 10.seconds)
|
||||
|
||||
// after merging with others
|
||||
replicator3 ! Get(KeyA, ReadAll(remainingOrDefault))
|
||||
val counter5 = expectMsgType[GetSuccess[GCounter]].dataValue
|
||||
|
|
|
|||
|
|
@ -105,6 +105,7 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
|
|||
expectMsg(ReplicaCount(5))
|
||||
}
|
||||
}
|
||||
enterBarrier("all-joined")
|
||||
|
||||
runOn(first) {
|
||||
for (_ <- 0 until 5) {
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import org.scalatest.Matchers
|
|||
import org.scalatest.WordSpec
|
||||
|
||||
object DeltaPropagationSelectorSpec {
|
||||
class TestSelector(val selfUniqueAddress: UniqueAddress, override val allNodes: Vector[Address])
|
||||
class TestSelector(val selfUniqueAddress: UniqueAddress, override val allNodes: Vector[UniqueAddress])
|
||||
extends DeltaPropagationSelector {
|
||||
override val gossipIntervalDivisor = 5
|
||||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
||||
|
|
@ -33,14 +33,14 @@ object DeltaPropagationSelectorSpec {
|
|||
|
||||
class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheckedTripleEquals {
|
||||
import DeltaPropagationSelectorSpec._
|
||||
val selfUniqueAddress = UniqueAddress(Address("akka", "Sys", "localhost", 4999), 1L)
|
||||
val nodes = (2500 until 2600).map(n => Address("akka", "Sys", "localhost", n)).toVector
|
||||
val selfUniqueAddress = UniqueAddress(Address("akka", "Sys", "localhost", 4999), 17L)
|
||||
val nodes = (2500 until 2600).map(n => UniqueAddress(Address("akka", "Sys", "localhost", n), 17L)).toVector
|
||||
|
||||
"DeltaPropagationSelector" must {
|
||||
"collect none when no nodes" in {
|
||||
val selector = new TestSelector(selfUniqueAddress, Vector.empty)
|
||||
selector.update("A", deltaA)
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
}
|
||||
|
|
@ -56,9 +56,9 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
DeltaPropagation(
|
||||
selfUniqueAddress,
|
||||
false,
|
||||
Map("A" -> Delta(DataEnvelope(deltaA), 1L, 1L), "B" -> Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||
Map("A" -> Delta(DataEnvelope(deltaA), 1L, 1L), "B" → Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
selector.hasDeltaEntries("B") should ===(false)
|
||||
|
|
@ -72,13 +72,13 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
DeltaPropagation(
|
||||
selfUniqueAddress,
|
||||
false,
|
||||
Map("A" -> Delta(DataEnvelope(deltaA), 1L, 1L), "B" -> Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected, nodes(1) -> expected))
|
||||
Map("A" -> Delta(DataEnvelope(deltaA), 1L, 1L), "B" → Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected, nodes(1) → expected))
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(true)
|
||||
selector.hasDeltaEntries("B") should ===(true)
|
||||
selector.collectPropagations() should ===(Map(nodes(2) -> expected))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
selector.hasDeltaEntries("B") should ===(false)
|
||||
|
|
@ -92,8 +92,8 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
DeltaPropagation(
|
||||
selfUniqueAddress,
|
||||
false,
|
||||
Map("A" -> Delta(DataEnvelope(deltaA), 1L, 1L), "B" -> Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected1, nodes(1) -> expected1))
|
||||
Map("A" -> Delta(DataEnvelope(deltaA), 1L, 1L), "B" → Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected1, nodes(1) → expected1))
|
||||
// new update before previous was propagated to all nodes
|
||||
selector.update("C", deltaC)
|
||||
val expected2 = DeltaPropagation(
|
||||
|
|
@ -103,14 +103,15 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
"A" -> Delta(DataEnvelope(deltaA), 1L, 1L),
|
||||
"B" -> Delta(DataEnvelope(deltaB), 1L, 1L),
|
||||
"C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||
val expected3 = DeltaPropagation(selfUniqueAddress, false, Map("C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(2) -> expected2, nodes(0) -> expected3))
|
||||
val expected3 =
|
||||
DeltaPropagation(selfUniqueAddress, false, Map("C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(2) -> expected2, nodes(0) → expected3))
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
selector.hasDeltaEntries("B") should ===(false)
|
||||
selector.hasDeltaEntries("C") should ===(true)
|
||||
selector.collectPropagations() should ===(Map(nodes(1) -> expected3))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("C") should ===(false)
|
||||
}
|
||||
|
|
@ -129,9 +130,10 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
selector.collectPropagations() should ===(Map(nodes(0) -> expected1))
|
||||
selector.update("A", delta3)
|
||||
selector.currentVersion("A") should ===(3L)
|
||||
val expected2 = DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta3), 3L, 3L)))
|
||||
val expected2 =
|
||||
DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta3), 3L, 3L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected2))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||
}
|
||||
|
||||
"merge deltas" in {
|
||||
|
|
@ -142,7 +144,8 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
override def nodesSliceSize(allNodesSize: Int): Int = 1
|
||||
}
|
||||
selector.update("A", delta1)
|
||||
val expected1 = DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta1), 1L, 1L)))
|
||||
val expected1 =
|
||||
DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta1), 1L, 1L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected1))
|
||||
|
||||
selector.update("A", delta2)
|
||||
|
|
@ -161,10 +164,11 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
|||
DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta2.merge(delta3)), 2L, 3L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected4))
|
||||
|
||||
val expected5 = DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta3), 3L, 3L)))
|
||||
val expected5 =
|
||||
DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta3), 3L, 3L)))
|
||||
selector.collectPropagations() should ===(Map(nodes(1) -> expected5))
|
||||
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||
}
|
||||
|
||||
"discard too large deltas" in {
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package akka.cluster.ddata
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorSelection
|
||||
|
|
@ -15,9 +16,11 @@ import akka.testkit._
|
|||
import akka.cluster.ddata.Replicator.Internal._
|
||||
import akka.cluster.ddata.Replicator._
|
||||
import akka.remote.RARP
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.UniqueAddress
|
||||
|
||||
object WriteAggregatorSpec {
|
||||
|
||||
val KeyA = GSetKey[String]("A")
|
||||
|
|
@ -26,41 +29,76 @@ object WriteAggregatorSpec {
|
|||
def writeAggregatorProps(
|
||||
data: GSet[String],
|
||||
consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
probes: Map[UniqueAddress, ActorRef],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
nodes: Set[UniqueAddress],
|
||||
unreachable: Set[UniqueAddress],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean): Props =
|
||||
Props(new TestWriteAggregator(KeyA, data, None, consistency, probes, nodes, unreachable, replyTo, durable))
|
||||
Props(
|
||||
new TestWriteAggregator(
|
||||
KeyA,
|
||||
data,
|
||||
None,
|
||||
consistency,
|
||||
probes,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
unreachable,
|
||||
replyTo,
|
||||
durable))
|
||||
|
||||
def writeAggregatorPropsWithDelta(
|
||||
data: ORSet[String],
|
||||
delta: Delta,
|
||||
consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
probes: Map[UniqueAddress, ActorRef],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
nodes: Set[UniqueAddress],
|
||||
unreachable: Set[UniqueAddress],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean): Props =
|
||||
Props(new TestWriteAggregator(KeyB, data, Some(delta), consistency, probes, nodes, unreachable, replyTo, durable))
|
||||
Props(
|
||||
new TestWriteAggregator(
|
||||
KeyB,
|
||||
data,
|
||||
Some(delta),
|
||||
consistency,
|
||||
probes,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
unreachable,
|
||||
replyTo,
|
||||
durable))
|
||||
|
||||
class TestWriteAggregator(
|
||||
key: Key.KeyR,
|
||||
data: ReplicatedData,
|
||||
delta: Option[Delta],
|
||||
consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
probes: Map[UniqueAddress, ActorRef],
|
||||
selfUniqueAddress: UniqueAddress,
|
||||
nodes: Set[UniqueAddress],
|
||||
unreachable: Set[UniqueAddress],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean)
|
||||
extends WriteAggregator(key, DataEnvelope(data), delta, consistency, None, nodes, unreachable, replyTo, durable) {
|
||||
extends WriteAggregator(
|
||||
key,
|
||||
DataEnvelope(data),
|
||||
delta,
|
||||
consistency,
|
||||
None,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
unreachable,
|
||||
replyTo,
|
||||
durable) {
|
||||
|
||||
override def replica(address: Address): ActorSelection =
|
||||
override def replica(address: UniqueAddress): ActorSelection =
|
||||
context.actorSelection(probes(address).path)
|
||||
|
||||
override def senderAddress(): Address =
|
||||
probes.find { case (_, r) => r == sender() }.get._1
|
||||
probes.find { case (_, r) => r == sender() }.get._1.address
|
||||
}
|
||||
|
||||
def writeAckAdapterProps(replica: ActorRef): Props =
|
||||
|
|
@ -105,10 +143,10 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
|
||||
else "akka.tcp"
|
||||
|
||||
val nodeA = Address(protocol, "Sys", "a", 2552)
|
||||
val nodeB = nodeA.copy(host = Some("b"))
|
||||
val nodeC = nodeA.copy(host = Some("c"))
|
||||
val nodeD = nodeA.copy(host = Some("d"))
|
||||
val nodeA = UniqueAddress(Address(protocol, "Sys", "a", 2552), 17L)
|
||||
val nodeB = UniqueAddress(Address(protocol, "Sys", "b", 2552), 17L)
|
||||
val nodeC = UniqueAddress(Address(protocol, "Sys", "c", 2552), 17L)
|
||||
val nodeD = UniqueAddress(Address(protocol, "Sys", "d", 2552), 17L)
|
||||
// 4 replicas + the local => 5
|
||||
val nodes = Set(nodeA, nodeB, nodeC, nodeD)
|
||||
|
||||
|
|
@ -118,13 +156,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
val writeMajority = WriteMajority(timeout)
|
||||
val writeAll = WriteAll(timeout)
|
||||
|
||||
def probes(probe: ActorRef): Map[Address, ActorRef] =
|
||||
val selfUniqueAddress: UniqueAddress = Cluster(system).selfUniqueAddress
|
||||
|
||||
def probes(probe: ActorRef): Map[UniqueAddress, ActorRef] =
|
||||
nodes.toSeq.map(_ -> system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
|
||||
|
||||
/**
|
||||
* Create a tuple for each node with the WriteAckAdapter and the TestProbe
|
||||
*/
|
||||
def probes(): Map[Address, TestMock] = {
|
||||
def probes(): Map[UniqueAddress, TestMock] = {
|
||||
nodes.toSeq.map(_ -> TestMock()).toMap
|
||||
}
|
||||
|
||||
|
|
@ -132,8 +172,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"send to at least N/2+1 replicas when WriteMajority" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeMajority,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
durable = false))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
|
|
@ -147,8 +194,16 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"send to more when no immediate reply" in {
|
||||
val testProbes = probes()
|
||||
val testProbeRefs = testProbes.map { case (a, tm) => a -> tm.writeAckAdapter }
|
||||
val aggr = system.actorOf(WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeMajority, testProbeRefs, nodes, Set(nodeC, nodeD), testActor, durable = false))
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeMajority,
|
||||
testProbeRefs,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set(nodeC, nodeD),
|
||||
testActor,
|
||||
durable = false))
|
||||
|
||||
testProbes(nodeA).expectMsgType[Write]
|
||||
// no reply
|
||||
|
|
@ -173,8 +228,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"timeout when less than required acks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeMajority,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
durable = false))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
|
|
@ -221,6 +283,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
delta,
|
||||
writeMajority,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
|
|
@ -244,6 +307,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
delta,
|
||||
writeAll,
|
||||
testProbeRefs,
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
|
|
@ -279,6 +343,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
delta,
|
||||
writeAll,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
|
|
@ -311,8 +376,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"not reply before local confirmation" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeThree,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
durable = true))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
|
|
@ -331,8 +403,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"tolerate WriteNack if enough WriteAck" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeThree,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
durable = true))
|
||||
|
||||
aggr ! UpdateSuccess(WriteAggregatorSpec.KeyA, None) // the local write
|
||||
probe.expectMsgType[Write]
|
||||
|
|
@ -350,8 +429,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"reply with StoreFailure when too many nacks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeMajority,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
durable = true))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteNack
|
||||
|
|
@ -371,8 +457,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"timeout when less than required acks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(
|
||||
WriteAggregatorSpec
|
||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
WriteAggregatorSpec.writeAggregatorProps(
|
||||
data,
|
||||
writeMajority,
|
||||
probes(probe.ref),
|
||||
selfUniqueAddress,
|
||||
nodes,
|
||||
Set.empty,
|
||||
testActor,
|
||||
durable = true))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
|
|
|
|||
|
|
@ -103,17 +103,26 @@ class ReplicatorMessageSerializerSpec
|
|||
pruning = Map(
|
||||
address1 -> PruningPerformed(System.currentTimeMillis()),
|
||||
address3 -> PruningInitialized(address2, Set(address1.address)))))
|
||||
checkSerialization(Write("A", DataEnvelope(data1)))
|
||||
checkSerialization(Write("A", DataEnvelope(data1), Some(address1)))
|
||||
checkSerialization(WriteAck)
|
||||
checkSerialization(WriteNack)
|
||||
checkSerialization(DeltaNack)
|
||||
checkSerialization(Read("A"))
|
||||
checkSerialization(Read("A", Some(address1)))
|
||||
checkSerialization(ReadResult(Some(DataEnvelope(data1))))
|
||||
checkSerialization(ReadResult(None))
|
||||
checkSerialization(
|
||||
Status(Map("A" -> ByteString.fromString("a"), "B" -> ByteString.fromString("b")), chunk = 3, totChunks = 10))
|
||||
Status(
|
||||
Map("A" -> ByteString.fromString("a"), "B" → ByteString.fromString("b")),
|
||||
chunk = 3,
|
||||
totChunks = 10,
|
||||
Some(17),
|
||||
Some(19)))
|
||||
checkSerialization(
|
||||
Gossip(Map("A" -> DataEnvelope(data1), "B" -> DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
||||
Gossip(
|
||||
Map("A" -> DataEnvelope(data1), "B" → DataEnvelope(GSet() + "b" + "c")),
|
||||
sendBack = true,
|
||||
Some(17),
|
||||
Some(19)))
|
||||
checkSerialization(
|
||||
DeltaPropagation(
|
||||
address1,
|
||||
|
|
@ -153,10 +162,10 @@ class ReplicatorMessageSerializerSpec
|
|||
|
||||
"get added element" in {
|
||||
val cache = new SmallCache[Read, String](2, 5.seconds, _ => null)
|
||||
val a = Read("a")
|
||||
val a = Read("a", Some(address1))
|
||||
cache.add(a, "A")
|
||||
cache.get(a) should be("A")
|
||||
val b = Read("b")
|
||||
val b = Read("b", Some(address1))
|
||||
cache.add(b, "B")
|
||||
cache.get(a) should be("A")
|
||||
cache.get(b) should be("B")
|
||||
|
|
@ -164,20 +173,20 @@ class ReplicatorMessageSerializerSpec
|
|||
|
||||
"return null for non-existing elements" in {
|
||||
val cache = new SmallCache[Read, String](4, 5.seconds, _ => null)
|
||||
val a = Read("a")
|
||||
val a = Read("a", Some(address1))
|
||||
cache.get(a) should be(null)
|
||||
cache.add(a, "A")
|
||||
val b = Read("b")
|
||||
val b = Read("b", Some(address1))
|
||||
cache.get(b) should be(null)
|
||||
}
|
||||
|
||||
"hold latest added elements" in {
|
||||
val cache = new SmallCache[Read, String](4, 5.seconds, _ => null)
|
||||
val a = Read("a")
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
val d = Read("d")
|
||||
val e = Read("e")
|
||||
val a = Read("a", Some(address1))
|
||||
val b = Read("b", Some(address1))
|
||||
val c = Read("c", Some(address1))
|
||||
val d = Read("d", Some(address1))
|
||||
val e = Read("e", Some(address1))
|
||||
cache.add(a, "A")
|
||||
cache.get(a) should be("A")
|
||||
cache.add(b, "B")
|
||||
|
|
@ -204,7 +213,7 @@ class ReplicatorMessageSerializerSpec
|
|||
|
||||
"handle Int wrap around" ignore { // ignored because it takes 20 seconds (but it works)
|
||||
val cache = new SmallCache[Read, String](2, 5.seconds, _ => null)
|
||||
val a = Read("a")
|
||||
val a = Read("a", Some(address1))
|
||||
val x = a -> "A"
|
||||
var n = 0
|
||||
while (n <= Int.MaxValue - 3) {
|
||||
|
|
@ -214,8 +223,8 @@ class ReplicatorMessageSerializerSpec
|
|||
|
||||
cache.get(a) should be("A")
|
||||
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
val b = Read("b", Some(address1))
|
||||
val c = Read("c", Some(address1))
|
||||
cache.add(b, "B")
|
||||
cache.get(a) should be("A")
|
||||
cache.get(b) should be("B")
|
||||
|
|
@ -241,7 +250,7 @@ class ReplicatorMessageSerializerSpec
|
|||
}
|
||||
|
||||
val cache = new SmallCache[Read, AnyRef](4, 5.seconds, a => createValue(a))
|
||||
val a = Read("a")
|
||||
val a = Read("a", Some(address1))
|
||||
val v1 = cache.getOrAdd(a)
|
||||
v1.toString should be("v1")
|
||||
(cache.getOrAdd(a) should be).theSameInstanceAs(v1)
|
||||
|
|
@ -249,8 +258,8 @@ class ReplicatorMessageSerializerSpec
|
|||
|
||||
"evict cache after time-to-live" in {
|
||||
val cache = new SmallCache[Read, AnyRef](4, 10.millis, _ => null)
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
val b = Read("b", Some(address1))
|
||||
val c = Read("c", Some(address1))
|
||||
cache.add(b, "B")
|
||||
cache.add(c, "C")
|
||||
|
||||
|
|
@ -262,8 +271,8 @@ class ReplicatorMessageSerializerSpec
|
|||
|
||||
"not evict cache before time-to-live" in {
|
||||
val cache = new SmallCache[Read, AnyRef](4, 5.seconds, _ => null)
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
val b = Read("b", Some(address1))
|
||||
val c = Read("c", Some(address1))
|
||||
cache.add(b, "B")
|
||||
cache.add(c, "C")
|
||||
cache.evict()
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue