Merge pull request #26421 from akka/wip-26284-removals-ClusterReceptionist-patriknw
Avoid false removals in ClusterReceptionist, #26284
This commit is contained in:
commit
7d58a1a2b5
16 changed files with 1714 additions and 396 deletions
|
|
@ -18,9 +18,15 @@ import akka.cluster.ddata.{ ORMultiMap, ORMultiMapKey, Replicator }
|
||||||
import akka.cluster.{ Cluster, ClusterEvent, UniqueAddress }
|
import akka.cluster.{ Cluster, ClusterEvent, UniqueAddress }
|
||||||
import akka.remote.AddressUidExtension
|
import akka.remote.AddressUidExtension
|
||||||
import akka.util.TypedMultiMap
|
import akka.util.TypedMultiMap
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.actor.Address
|
||||||
|
import akka.cluster.ClusterEvent.ClusterDomainEvent
|
||||||
|
import akka.cluster.ClusterEvent.ClusterShuttingDown
|
||||||
|
import akka.cluster.ClusterEvent.MemberJoined
|
||||||
|
import akka.cluster.ClusterEvent.MemberUp
|
||||||
|
import akka.cluster.ClusterEvent.MemberWeaklyUp
|
||||||
|
|
||||||
// just to provide a log class
|
// just to provide a log class
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
@InternalApi
|
@InternalApi
|
||||||
|
|
@ -39,16 +45,19 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
// values contain system uid to make it possible to discern actors at the same
|
// values contain system uid to make it possible to discern actors at the same
|
||||||
// path in different incarnations of a cluster node
|
// path in different incarnations of a cluster node
|
||||||
final case class Entry(ref: ActorRef[_], systemUid: Long) {
|
final case class Entry(ref: ActorRef[_], systemUid: Long) {
|
||||||
def uniqueAddress(selfUniqueAddress: UniqueAddress): UniqueAddress =
|
def uniqueAddress(selfAddress: Address): UniqueAddress =
|
||||||
if (ref.path.address.hasLocalScope) selfUniqueAddress
|
if (ref.path.address.hasLocalScope) UniqueAddress(selfAddress, systemUid)
|
||||||
else UniqueAddress(ref.path.address, systemUid)
|
else UniqueAddress(ref.path.address, systemUid)
|
||||||
override def toString = ref.path.toString + "#" + ref.path.uid
|
override def toString: String =
|
||||||
|
s"${ref.path.toString}#${ref.path.uid} @ $systemUid"
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private sealed trait InternalCommand extends Command
|
private sealed trait InternalCommand extends Command
|
||||||
private final case class RegisteredActorTerminated[T](key: ServiceKey[T], ref: ActorRef[T]) extends InternalCommand
|
private final case class RegisteredActorTerminated[T](key: ServiceKey[T], ref: ActorRef[T]) extends InternalCommand
|
||||||
private final case class SubscriberTerminated[T](key: ServiceKey[T], ref: ActorRef[ReceptionistMessages.Listing[T]])
|
private final case class SubscriberTerminated[T](key: ServiceKey[T], ref: ActorRef[ReceptionistMessages.Listing[T]])
|
||||||
extends InternalCommand
|
extends InternalCommand
|
||||||
|
private final case class NodeAdded(addresses: UniqueAddress) extends InternalCommand
|
||||||
private final case class NodeRemoved(addresses: UniqueAddress) extends InternalCommand
|
private final case class NodeRemoved(addresses: UniqueAddress) extends InternalCommand
|
||||||
private final case class ChangeFromReplicator(key: DDataKey, value: ORMultiMap[ServiceKey[_], Entry])
|
private final case class ChangeFromReplicator(key: DDataKey, value: ORMultiMap[ServiceKey[_], Entry])
|
||||||
extends InternalCommand
|
extends InternalCommand
|
||||||
|
|
@ -76,7 +85,8 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
ctx.setLoggerClass(classOf[ClusterReceptionist])
|
ctx.setLoggerClass(classOf[ClusterReceptionist])
|
||||||
Behaviors.withTimers { timers =>
|
Behaviors.withTimers { timers =>
|
||||||
val setup = new Setup(ctx)
|
val setup = new Setup(ctx)
|
||||||
val registry = ShardedServiceRegistry(setup.settings.distributedKeyCount)
|
// include selfUniqueAddress so that it can be used locally before joining cluster
|
||||||
|
val registry = ShardedServiceRegistry(setup.settings.distributedKeyCount).addNode(setup.selfUniqueAddress)
|
||||||
|
|
||||||
// subscribe to changes from other nodes
|
// subscribe to changes from other nodes
|
||||||
val replicatorMessageAdapter: ActorRef[Replicator.ReplicatorMessage] =
|
val replicatorMessageAdapter: ActorRef[Replicator.ReplicatorMessage] =
|
||||||
|
|
@ -90,13 +100,26 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
registry.allDdataKeys.foreach(key =>
|
registry.allDdataKeys.foreach(key =>
|
||||||
setup.replicator ! Replicator.Subscribe(key, replicatorMessageAdapter.toUntyped))
|
setup.replicator ! Replicator.Subscribe(key, replicatorMessageAdapter.toUntyped))
|
||||||
|
|
||||||
|
// keep track of cluster members
|
||||||
// remove entries when members are removed
|
// remove entries when members are removed
|
||||||
val clusterEventMessageAdapter: ActorRef[MemberRemoved] =
|
val clusterEventMessageAdapter: ActorRef[ClusterDomainEvent] =
|
||||||
ctx.messageAdapter[MemberRemoved] { case MemberRemoved(member, _) => NodeRemoved(member.uniqueAddress) }
|
ctx.messageAdapter[ClusterDomainEvent] {
|
||||||
|
case MemberJoined(member) => NodeAdded(member.uniqueAddress)
|
||||||
|
case MemberWeaklyUp(member) => NodeAdded(member.uniqueAddress)
|
||||||
|
case MemberUp(member) => NodeAdded(member.uniqueAddress)
|
||||||
|
case MemberRemoved(member, _) => NodeRemoved(member.uniqueAddress)
|
||||||
|
case ClusterShuttingDown => NodeRemoved(setup.cluster.selfUniqueAddress)
|
||||||
|
case other =>
|
||||||
|
throw new IllegalStateException(s"Unexpected ClusterDomainEvent $other. Please report bug.")
|
||||||
|
}
|
||||||
setup.cluster.subscribe(
|
setup.cluster.subscribe(
|
||||||
clusterEventMessageAdapter.toUntyped,
|
clusterEventMessageAdapter.toUntyped,
|
||||||
ClusterEvent.InitialStateAsEvents,
|
ClusterEvent.InitialStateAsEvents,
|
||||||
classOf[MemberRemoved])
|
classOf[MemberJoined],
|
||||||
|
classOf[MemberWeaklyUp],
|
||||||
|
classOf[MemberUp],
|
||||||
|
classOf[MemberRemoved],
|
||||||
|
ClusterShuttingDown.getClass)
|
||||||
|
|
||||||
// also periodic cleanup in case removal from ORMultiMap is skipped due to concurrent update,
|
// also periodic cleanup in case removal from ORMultiMap is skipped due to concurrent update,
|
||||||
// which is possible for OR CRDTs - done with an adapter to leverage the existing NodesRemoved message
|
// which is possible for OR CRDTs - done with an adapter to leverage the existing NodesRemoved message
|
||||||
|
|
@ -129,28 +152,21 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
def watchWith(ctx: ActorContext[Command], target: ActorRef[_], msg: InternalCommand): Unit =
|
def watchWith(ctx: ActorContext[Command], target: ActorRef[_], msg: InternalCommand): Unit =
|
||||||
ctx.spawnAnonymous[Nothing](Behaviors.setup[Nothing] { innerCtx =>
|
ctx.spawnAnonymous[Nothing](Behaviors.setup[Nothing] { innerCtx =>
|
||||||
innerCtx.watch(target)
|
innerCtx.watch(target)
|
||||||
Behaviors.receive[Nothing]((_, _) => Behaviors.same).receiveSignal {
|
Behaviors.receiveSignal[Nothing] {
|
||||||
case (_, Terminated(`target`)) =>
|
case (_, Terminated(`target`)) =>
|
||||||
ctx.self ! msg
|
ctx.self ! msg
|
||||||
Behaviors.stopped
|
Behaviors.stopped
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
def notifySubscribersFor(key: AbstractServiceKey, state: ServiceRegistry): Unit = {
|
def isLeader = {
|
||||||
// filter tombstoned refs to avoid an extra update
|
cluster.state.leader.contains(cluster.selfAddress)
|
||||||
// to subscribers in the case of lost removals (because of how ORMultiMap works)
|
|
||||||
val refsForKey = state.actorRefsFor(key)
|
|
||||||
val refsWithoutTombstoned =
|
|
||||||
if (registry.tombstones.isEmpty) refsForKey
|
|
||||||
else refsForKey.filterNot(registry.hasTombstone)
|
|
||||||
val msg = ReceptionistMessages.Listing(key.asServiceKey, refsWithoutTombstoned)
|
|
||||||
subscriptions.get(key).foreach(_ ! msg)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def nodesRemoved(addresses: Set[UniqueAddress]): Behavior[Command] = {
|
def nodesRemoved(addresses: Set[UniqueAddress]): Unit = {
|
||||||
// ok to update from several nodes but more efficient to try to do it from one node
|
// ok to update from several nodes but more efficient to try to do it from one node
|
||||||
if (cluster.state.leader.contains(cluster.selfAddress) && addresses.nonEmpty) {
|
if (isLeader) {
|
||||||
def isOnRemovedNode(entry: Entry): Boolean = addresses(entry.uniqueAddress(setup.selfUniqueAddress))
|
def isOnRemovedNode(entry: Entry): Boolean = addresses(entry.uniqueAddress(setup.selfUniqueAddress.address))
|
||||||
|
|
||||||
val removals = {
|
val removals = {
|
||||||
registry.allServices.foldLeft(Map.empty[AbstractServiceKey, Set[Entry]]) {
|
registry.allServices.foldLeft(Map.empty[AbstractServiceKey, Set[Entry]]) {
|
||||||
|
|
@ -164,7 +180,8 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
if (removals.nonEmpty) {
|
if (removals.nonEmpty) {
|
||||||
if (ctx.log.isDebugEnabled)
|
if (ctx.log.isDebugEnabled)
|
||||||
ctx.log.debug(
|
ctx.log.debug(
|
||||||
"Node(s) [{}] removed, updating registry removing: [{}]",
|
"ClusterReceptionist [{}] - Node(s) removed [{}], updating registry removing entries: [{}]",
|
||||||
|
cluster.selfAddress,
|
||||||
addresses.mkString(","),
|
addresses.mkString(","),
|
||||||
removals
|
removals
|
||||||
.map {
|
.map {
|
||||||
|
|
@ -185,33 +202,38 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
Behaviors.same
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def onCommand(cmd: Command): Behavior[Command] = cmd match {
|
def onCommand(cmd: Command): Behavior[Command] = cmd match {
|
||||||
case ReceptionistMessages.Register(key, serviceInstance, maybeReplyTo) =>
|
case ReceptionistMessages.Register(key, serviceInstance, maybeReplyTo) =>
|
||||||
val entry = Entry(serviceInstance, setup.selfSystemUid)
|
if (serviceInstance.path.address.hasLocalScope) {
|
||||||
ctx.log.debug("Actor was registered: [{}] [{}]", key, entry)
|
val entry = Entry(serviceInstance, setup.selfSystemUid)
|
||||||
watchWith(ctx, serviceInstance, RegisteredActorTerminated(key, serviceInstance))
|
ctx.log.debug("ClusterReceptionist [{}] - Actor was registered: [{}] [{}]", cluster.selfAddress, key, entry)
|
||||||
maybeReplyTo match {
|
watchWith(ctx, serviceInstance, RegisteredActorTerminated(key, serviceInstance))
|
||||||
case Some(replyTo) => replyTo ! ReceptionistMessages.Registered(key, serviceInstance)
|
maybeReplyTo match {
|
||||||
case None =>
|
case Some(replyTo) => replyTo ! ReceptionistMessages.Registered(key, serviceInstance)
|
||||||
}
|
case None =>
|
||||||
val ddataKey = registry.ddataKeyFor(key)
|
}
|
||||||
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
|
val ddataKey = registry.ddataKeyFor(key)
|
||||||
ServiceRegistry(registry).addBinding(key, entry).toORMultiMap
|
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
|
||||||
|
ServiceRegistry(registry).addBinding(key, entry).toORMultiMap
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
ctx.log.error(s"ClusterReceptionist [{}] - Register of non-local [{}] is not supported", serviceInstance)
|
||||||
}
|
}
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
|
|
||||||
case ReceptionistMessages.Find(key, replyTo) =>
|
case ReceptionistMessages.Find(key, replyTo) =>
|
||||||
replyTo ! ReceptionistMessages.Listing(key.asServiceKey, registry.actorRefsFor(key))
|
replyTo ! ReceptionistMessages.Listing(key.asServiceKey, registry.activeActorRefsFor(key, selfUniqueAddress))
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
|
|
||||||
case ReceptionistMessages.Subscribe(key, subscriber) =>
|
case ReceptionistMessages.Subscribe(key, subscriber) =>
|
||||||
watchWith(ctx, subscriber, SubscriberTerminated(key, subscriber))
|
watchWith(ctx, subscriber, SubscriberTerminated(key, subscriber))
|
||||||
|
|
||||||
// immediately reply with initial listings to the new subscriber
|
// immediately reply with initial listings to the new subscriber
|
||||||
subscriber ! ReceptionistMessages.Listing(key.asServiceKey, registry.actorRefsFor(key))
|
val listing =
|
||||||
|
ReceptionistMessages.Listing(key.asServiceKey, registry.activeActorRefsFor(key, selfUniqueAddress))
|
||||||
|
subscriber ! listing
|
||||||
|
|
||||||
next(newSubscriptions = subscriptions.inserted(key)(subscriber))
|
next(newSubscriptions = subscriptions.inserted(key)(subscriber))
|
||||||
}
|
}
|
||||||
|
|
@ -223,7 +245,11 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
|
|
||||||
case RegisteredActorTerminated(key, serviceInstance) =>
|
case RegisteredActorTerminated(key, serviceInstance) =>
|
||||||
val entry = Entry(serviceInstance, setup.selfSystemUid)
|
val entry = Entry(serviceInstance, setup.selfSystemUid)
|
||||||
ctx.log.debug("Registered actor terminated: [{}] [{}]", key.asServiceKey.id, entry)
|
ctx.log.debug(
|
||||||
|
"ClusterReceptionist [{}] - Registered actor terminated: [{}] [{}]",
|
||||||
|
cluster.selfAddress,
|
||||||
|
key.asServiceKey.id,
|
||||||
|
entry)
|
||||||
val ddataKey = registry.ddataKeyFor(key)
|
val ddataKey = registry.ddataKeyFor(key)
|
||||||
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
|
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
|
||||||
ServiceRegistry(registry).removeBinding(key, entry).toORMultiMap
|
ServiceRegistry(registry).removeBinding(key, entry).toORMultiMap
|
||||||
|
|
@ -240,25 +266,42 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
if (changedKeys.nonEmpty) {
|
if (changedKeys.nonEmpty) {
|
||||||
if (ctx.log.isDebugEnabled) {
|
if (ctx.log.isDebugEnabled) {
|
||||||
ctx.log.debug(
|
ctx.log.debug(
|
||||||
"Change from replicator: [{}], changes: [{}], tombstones [{}]",
|
"ClusterReceptionist [{}] - Change from replicator: [{}], changes: [{}], tombstones [{}]",
|
||||||
|
cluster.selfAddress,
|
||||||
newState.entries.entries,
|
newState.entries.entries,
|
||||||
changedKeys
|
changedKeys
|
||||||
.map(key => key.asServiceKey.id -> newState.entriesFor(key).mkString("[", ", ", "]"))
|
.map(key => key.asServiceKey.id -> newState.entriesFor(key).mkString("[", ", ", "]"))
|
||||||
.mkString(", "),
|
.mkString(", "),
|
||||||
registry.tombstones.mkString(", "))
|
newRegistry.tombstones.mkString(", "))
|
||||||
}
|
}
|
||||||
|
|
||||||
changedKeys.foreach { changedKey =>
|
changedKeys.foreach { changedKey =>
|
||||||
notifySubscribersFor(changedKey, newState)
|
val serviceKey = changedKey.asServiceKey
|
||||||
|
|
||||||
|
val subscribers = subscriptions.get(changedKey)
|
||||||
|
if (subscribers.nonEmpty) {
|
||||||
|
val listing =
|
||||||
|
ReceptionistMessages
|
||||||
|
.Listing(serviceKey, newRegistry.activeActorRefsFor(serviceKey, selfUniqueAddress))
|
||||||
|
subscribers.foreach(_ ! listing)
|
||||||
|
}
|
||||||
|
|
||||||
// because of how ORMultiMap/ORset works, we could have a case where an actor we removed
|
// because of how ORMultiMap/ORset works, we could have a case where an actor we removed
|
||||||
// is re-introduced because of a concurrent update, in that case we need to re-remove it
|
// is re-introduced because of a concurrent update, in that case we need to re-remove it
|
||||||
val serviceKey = changedKey.asServiceKey
|
val tombstonedButReAdded = newRegistry.actorRefsFor(serviceKey).filter(newRegistry.hasTombstone)
|
||||||
val tombstonedButReAdded =
|
if (tombstonedButReAdded.nonEmpty) {
|
||||||
newRegistry.actorRefsFor(serviceKey).filter(newRegistry.hasTombstone)
|
if (ctx.log.isDebugEnabled)
|
||||||
tombstonedButReAdded.foreach { actorRef =>
|
ctx.log.debug(
|
||||||
ctx.log.debug("Saw actorref that was tomstoned {}, re-removing.", actorRef)
|
"ClusterReceptionist [{}] - Saw ActorRefs that were tomstoned [{}], re-removing.",
|
||||||
|
cluster.selfAddress,
|
||||||
|
tombstonedButReAdded.mkString(", "))
|
||||||
|
|
||||||
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
|
replicator ! Replicator.Update(ddataKey, EmptyORMultiMap, settings.writeConsistency) { registry =>
|
||||||
ServiceRegistry(registry).removeBinding(serviceKey, Entry(actorRef, setup.selfSystemUid)).toORMultiMap
|
tombstonedButReAdded
|
||||||
|
.foldLeft(ServiceRegistry(registry)) { (acc, ref) =>
|
||||||
|
acc.removeBinding(serviceKey, Entry(ref, setup.selfSystemUid))
|
||||||
|
}
|
||||||
|
.toORMultiMap
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -268,34 +311,51 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
}
|
}
|
||||||
|
|
||||||
|
case NodeAdded(uniqueAddress) =>
|
||||||
|
next(registry.addNode(uniqueAddress))
|
||||||
|
|
||||||
case NodeRemoved(uniqueAddress) =>
|
case NodeRemoved(uniqueAddress) =>
|
||||||
// ok to update from several nodes but more efficient to try to do it from one node
|
if (uniqueAddress == selfUniqueAddress) {
|
||||||
if (cluster.state.leader.contains(cluster.selfAddress)) {
|
ctx.log.debug("ClusterReceptionist [{}] - terminated/removed", cluster.selfAddress)
|
||||||
ctx.log.debug(s"Leader node observed removed address [{}]", uniqueAddress)
|
// If self cluster node is shutting down our own entries should have been removed via
|
||||||
nodesRemoved(Set(uniqueAddress))
|
// watch-Terminated or will be removed by other nodes. This point is anyway too late.
|
||||||
} else Behaviors.same
|
Behaviors.stopped
|
||||||
|
} else {
|
||||||
|
// Ok to update from several nodes but more efficient to try to do it from one node.
|
||||||
|
if (isLeader) {
|
||||||
|
ctx.log.debug(
|
||||||
|
"ClusterReceptionist [{}] - Leader node observed removed node [{}]",
|
||||||
|
cluster.selfAddress,
|
||||||
|
uniqueAddress)
|
||||||
|
nodesRemoved(Set(uniqueAddress))
|
||||||
|
}
|
||||||
|
|
||||||
|
next(registry.removeNode(uniqueAddress))
|
||||||
|
}
|
||||||
|
|
||||||
case RemoveTick =>
|
case RemoveTick =>
|
||||||
// ok to update from several nodes but more efficient to try to do it from one node
|
// ok to update from several nodes but more efficient to try to do it from one node
|
||||||
if (cluster.state.leader.contains(cluster.selfAddress)) {
|
if (isLeader) {
|
||||||
val allAddressesInState: Set[UniqueAddress] = registry.allUniqueAddressesInState(setup.selfUniqueAddress)
|
val allAddressesInState: Set[UniqueAddress] = registry.allUniqueAddressesInState(setup.selfUniqueAddress)
|
||||||
val clusterAddresses = cluster.state.members.map(_.uniqueAddress)
|
val notInCluster = allAddressesInState.diff(registry.nodes)
|
||||||
val notInCluster = allAddressesInState.diff(clusterAddresses)
|
|
||||||
|
|
||||||
if (notInCluster.isEmpty) Behavior.same
|
if (notInCluster.isEmpty) Behavior.same
|
||||||
else {
|
else {
|
||||||
if (ctx.log.isDebugEnabled)
|
if (ctx.log.isDebugEnabled)
|
||||||
ctx.log.debug("Leader node cleanup tick, removed nodes: [{}]", notInCluster.mkString(","))
|
ctx.log.debug(
|
||||||
|
"ClusterReceptionist [{}] - Leader node cleanup tick, removed nodes: [{}]",
|
||||||
|
cluster.selfAddress,
|
||||||
|
notInCluster.mkString(","))
|
||||||
nodesRemoved(notInCluster)
|
nodesRemoved(notInCluster)
|
||||||
}
|
}
|
||||||
} else
|
}
|
||||||
Behavior.same
|
Behavior.same
|
||||||
|
|
||||||
case PruneTombstonesTick =>
|
case PruneTombstonesTick =>
|
||||||
val prunedRegistry = registry.pruneTombstones()
|
val prunedRegistry = registry.pruneTombstones()
|
||||||
if (prunedRegistry eq registry) Behaviors.same
|
if (prunedRegistry eq registry) Behaviors.same
|
||||||
else {
|
else {
|
||||||
ctx.log.debug(s"Pruning tombstones")
|
ctx.log.debug("ClusterReceptionist [{}] - Pruning tombstones", cluster.selfAddress)
|
||||||
next(prunedRegistry)
|
next(prunedRegistry)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -309,4 +369,5 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -23,23 +23,26 @@ import scala.concurrent.duration.Deadline
|
||||||
val key = ORMultiMapKey[ServiceKey[_], Entry](s"ReceptionistKey_$n")
|
val key = ORMultiMapKey[ServiceKey[_], Entry](s"ReceptionistKey_$n")
|
||||||
key -> new ServiceRegistry(EmptyORMultiMap)
|
key -> new ServiceRegistry(EmptyORMultiMap)
|
||||||
}.toMap
|
}.toMap
|
||||||
new ShardedServiceRegistry(emptyRegistries, Map.empty)
|
new ShardedServiceRegistry(emptyRegistries, Map.empty, Set.empty)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*
|
||||||
* Two level structure for keeping service registry to be able to shard entries over multiple ddata keys (to not
|
* Two level structure for keeping service registry to be able to shard entries over multiple ddata keys (to not
|
||||||
* get too large ddata messages)
|
* get too large ddata messages)
|
||||||
*
|
*
|
||||||
* @param tombstones Local actors that were stopped and should not be re-added to the available set of actors
|
* @param tombstones Local actors that were stopped and should not be re-added to the available set of actors
|
||||||
* for a key. Since the only way to unregister is to stop, we don't need to keep track of
|
* for a key. Since the only way to unregister is to stop, we don't need to keep track of
|
||||||
* the service key
|
* the service key
|
||||||
* INTERNAL API
|
*
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] final case class ShardedServiceRegistry(
|
@InternalApi private[akka] final case class ShardedServiceRegistry(
|
||||||
serviceRegistries: Map[DDataKey, ServiceRegistry],
|
serviceRegistries: Map[DDataKey, ServiceRegistry],
|
||||||
tombstones: Map[ActorRef[_], Deadline]) {
|
tombstones: Map[ActorRef[_], Deadline],
|
||||||
|
nodes: Set[UniqueAddress]) {
|
||||||
|
|
||||||
private val keys = serviceRegistries.keySet.toArray
|
private val keys = serviceRegistries.keySet.toArray
|
||||||
|
|
||||||
|
|
@ -56,22 +59,32 @@ import scala.concurrent.duration.Deadline
|
||||||
def allEntries: Iterator[Entry] = allServices.flatMap(_._2)
|
def allEntries: Iterator[Entry] = allServices.flatMap(_._2)
|
||||||
|
|
||||||
def actorRefsFor[T](key: ServiceKey[T]): Set[ActorRef[T]] = {
|
def actorRefsFor[T](key: ServiceKey[T]): Set[ActorRef[T]] = {
|
||||||
val dDataKey = ddataKeyFor(key)
|
val ddataKey = ddataKeyFor(key)
|
||||||
serviceRegistries(dDataKey).actorRefsFor(key)
|
serviceRegistries(ddataKey).actorRefsFor(key)
|
||||||
}
|
}
|
||||||
|
|
||||||
def withServiceRegistry(dDataKey: DDataKey, registry: ServiceRegistry): ShardedServiceRegistry =
|
def activeActorRefsFor[T](key: ServiceKey[T], selfUniqueAddress: UniqueAddress): Set[ActorRef[T]] = {
|
||||||
copy(serviceRegistries + (dDataKey -> registry), tombstones)
|
val ddataKey = ddataKeyFor(key)
|
||||||
|
val entries = serviceRegistries(ddataKey).entriesFor(key)
|
||||||
|
val selfAddress = selfUniqueAddress.address
|
||||||
|
entries.collect {
|
||||||
|
case entry if nodes.contains(entry.uniqueAddress(selfAddress)) && !hasTombstone(entry.ref) =>
|
||||||
|
entry.ref.asInstanceOf[ActorRef[key.Protocol]]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def withServiceRegistry(ddataKey: DDataKey, registry: ServiceRegistry): ShardedServiceRegistry =
|
||||||
|
copy(serviceRegistries + (ddataKey -> registry), tombstones)
|
||||||
|
|
||||||
def allUniqueAddressesInState(selfUniqueAddress: UniqueAddress): Set[UniqueAddress] =
|
def allUniqueAddressesInState(selfUniqueAddress: UniqueAddress): Set[UniqueAddress] =
|
||||||
allEntries.collect {
|
allEntries.collect {
|
||||||
// we don't care about local (empty host:port addresses)
|
// we don't care about local (empty host:port addresses)
|
||||||
case entry if entry.ref.path.address.hasGlobalScope =>
|
case entry if entry.ref.path.address.hasGlobalScope =>
|
||||||
entry.uniqueAddress(selfUniqueAddress)
|
entry.uniqueAddress(selfUniqueAddress.address)
|
||||||
}.toSet
|
}.toSet
|
||||||
|
|
||||||
def collectChangedKeys(dDataKey: DDataKey, newRegistry: ServiceRegistry): Set[AbstractServiceKey] = {
|
def collectChangedKeys(ddataKey: DDataKey, newRegistry: ServiceRegistry): Set[AbstractServiceKey] = {
|
||||||
val previousRegistry = registryFor(dDataKey)
|
val previousRegistry = registryFor(ddataKey)
|
||||||
ServiceRegistry.collectChangedKeys(previousRegistry, newRegistry)
|
ServiceRegistry.collectChangedKeys(previousRegistry, newRegistry)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -88,7 +101,7 @@ import scala.concurrent.duration.Deadline
|
||||||
copy(tombstones = tombstones + (actorRef -> deadline))
|
copy(tombstones = tombstones + (actorRef -> deadline))
|
||||||
|
|
||||||
def hasTombstone(actorRef: ActorRef[_]): Boolean =
|
def hasTombstone(actorRef: ActorRef[_]): Boolean =
|
||||||
tombstones.contains(actorRef)
|
tombstones.nonEmpty && tombstones.contains(actorRef)
|
||||||
|
|
||||||
def pruneTombstones(): ShardedServiceRegistry = {
|
def pruneTombstones(): ShardedServiceRegistry = {
|
||||||
copy(tombstones = tombstones.filter {
|
copy(tombstones = tombstones.filter {
|
||||||
|
|
@ -96,6 +109,12 @@ import scala.concurrent.duration.Deadline
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def addNode(node: UniqueAddress): ShardedServiceRegistry =
|
||||||
|
copy(nodes = nodes + node)
|
||||||
|
|
||||||
|
def removeNode(node: UniqueAddress): ShardedServiceRegistry =
|
||||||
|
copy(nodes = nodes - node)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -18,10 +18,13 @@ import akka.actor.testkit.typed.FishingOutcome
|
||||||
import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, FishingOutcomes, TestProbe }
|
import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, FishingOutcomes, TestProbe }
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.{ Matchers, WordSpec }
|
import org.scalatest.{ Matchers, WordSpec }
|
||||||
|
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.cluster.typed.Down
|
||||||
|
import akka.cluster.typed.JoinSeedNodes
|
||||||
|
import akka.cluster.typed.Leave
|
||||||
|
|
||||||
object ClusterReceptionistSpec {
|
object ClusterReceptionistSpec {
|
||||||
val config = ConfigFactory.parseString(s"""
|
val config = ConfigFactory.parseString(s"""
|
||||||
akka.loglevel = DEBUG # issue #24960
|
akka.loglevel = DEBUG # issue #24960
|
||||||
|
|
@ -42,9 +45,17 @@ object ClusterReceptionistSpec {
|
||||||
akka.remote.netty.tcp.host = 127.0.0.1
|
akka.remote.netty.tcp.host = 127.0.0.1
|
||||||
akka.remote.artery.canonical.port = 0
|
akka.remote.artery.canonical.port = 0
|
||||||
akka.remote.artery.canonical.hostname = 127.0.0.1
|
akka.remote.artery.canonical.hostname = 127.0.0.1
|
||||||
|
|
||||||
|
akka.remote.retry-gate-closed-for = 1 s
|
||||||
|
|
||||||
|
akka.cluster.typed.receptionist {
|
||||||
|
pruning-interval = 1 s
|
||||||
|
}
|
||||||
|
|
||||||
akka.cluster {
|
akka.cluster {
|
||||||
auto-down-unreachable-after = 0s
|
#auto-down-unreachable-after = 0s
|
||||||
jmx.multi-mbeans-in-same-jvm = on
|
jmx.multi-mbeans-in-same-jvm = on
|
||||||
|
failure-detector.acceptable-heartbeat-pause = 3s
|
||||||
}
|
}
|
||||||
""")
|
""")
|
||||||
|
|
||||||
|
|
@ -109,7 +120,7 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
val regProbe1 = TestProbe[Any]()(system1)
|
val regProbe1 = TestProbe[Any]()(system1)
|
||||||
val regProbe2 = TestProbe[Any]()(system2)
|
val regProbe2 = TestProbe[Any]()(system2)
|
||||||
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) == 2)
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2), 10.seconds)
|
||||||
|
|
||||||
system2.receptionist ! Subscribe(PingKey, regProbe2.ref)
|
system2.receptionist ! Subscribe(PingKey, regProbe2.ref)
|
||||||
regProbe2.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
regProbe2.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
@ -132,7 +143,15 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"remove registrations when node dies" in {
|
"remove registrations when node dies" in {
|
||||||
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-2", ClusterReceptionistSpec.config)
|
testNodeRemoval(down = true)
|
||||||
|
}
|
||||||
|
|
||||||
|
"remove registrations when node leaves" in {
|
||||||
|
testNodeRemoval(down = false)
|
||||||
|
}
|
||||||
|
|
||||||
|
def testNodeRemoval(down: Boolean): Unit = {
|
||||||
|
val testKit1 = ActorTestKit(s"ClusterReceptionistSpec-test-3-$down", ClusterReceptionistSpec.config)
|
||||||
val system1 = testKit1.system
|
val system1 = testKit1.system
|
||||||
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
||||||
val system2 = testKit2.system
|
val system2 = testKit2.system
|
||||||
|
|
@ -146,72 +165,47 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
val regProbe1 = TestProbe[Any]()(system1)
|
val regProbe1 = TestProbe[Any]()(system1)
|
||||||
val regProbe2 = TestProbe[Any]()(system2)
|
val regProbe2 = TestProbe[Any]()(system2)
|
||||||
|
|
||||||
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) == 2)
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2), 10.seconds)
|
||||||
|
|
||||||
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
||||||
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
||||||
|
val service1 = testKit1.spawn(pingPongBehavior)
|
||||||
|
system1.receptionist ! Register(PingKey, service1, regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Registered(PingKey, service1))
|
||||||
|
|
||||||
|
regProbe1.expectMessage(Listing(PingKey, Set(service1)))
|
||||||
|
|
||||||
val service2 = testKit2.spawn(pingPongBehavior)
|
val service2 = testKit2.spawn(pingPongBehavior)
|
||||||
system2.receptionist ! Register(PingKey, service2, regProbe2.ref)
|
system2.receptionist ! Register(PingKey, service2, regProbe2.ref)
|
||||||
regProbe2.expectMessage(Registered(PingKey, service2))
|
regProbe2.expectMessage(Registered(PingKey, service2))
|
||||||
|
|
||||||
val remoteServiceRefs = regProbe1.expectMessageType[Listing].serviceInstances(PingKey)
|
val serviceRefs2 = regProbe1.expectMessageType[Listing].serviceInstances(PingKey)
|
||||||
val theRef = remoteServiceRefs.head
|
serviceRefs2.size should ===(2)
|
||||||
theRef ! Ping(regProbe1.ref)
|
|
||||||
regProbe1.expectMessage(Pong)
|
if (down) {
|
||||||
|
// abrupt termination
|
||||||
|
Await.ready(system2.terminate(), 10.seconds)
|
||||||
|
clusterNode1.manager ! Down(clusterNode2.selfMember.address)
|
||||||
|
} else {
|
||||||
|
clusterNode1.manager ! Leave(clusterNode2.selfMember.address)
|
||||||
|
}
|
||||||
|
|
||||||
|
regProbe1.expectMessage(10.seconds, Listing(PingKey, Set(service1)))
|
||||||
|
|
||||||
|
// register another after removal
|
||||||
|
val service1b = testKit1.spawn(pingPongBehavior)
|
||||||
|
system1.receptionist ! Register(PingKey, service1b, regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Registered(PingKey, service1b))
|
||||||
|
regProbe1.expectMessage(Listing(PingKey, Set(service1, service1b)))
|
||||||
|
|
||||||
// abrupt termination
|
|
||||||
system2.terminate()
|
|
||||||
regProbe1.expectMessage(10.seconds, Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
|
||||||
} finally {
|
} finally {
|
||||||
testKit1.shutdownTestKit()
|
testKit1.shutdownTestKit()
|
||||||
testKit2.shutdownTestKit()
|
testKit2.shutdownTestKit()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"work with services registered before node joins cluster" in {
|
"not remove registrations when self is shutdown" in {
|
||||||
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-3", ClusterReceptionistSpec.config)
|
|
||||||
val system1 = testKit1.system
|
|
||||||
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
|
||||||
val system2 = testKit2.system
|
|
||||||
try {
|
|
||||||
|
|
||||||
val clusterNode1 = Cluster(system1)
|
|
||||||
clusterNode1.manager ! Join(clusterNode1.selfMember.address)
|
|
||||||
|
|
||||||
val regProbe1 = TestProbe[Any]()(system1)
|
|
||||||
val regProbe2 = TestProbe[Any]()(system2)
|
|
||||||
|
|
||||||
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) == 2)
|
|
||||||
|
|
||||||
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
|
||||||
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
|
||||||
|
|
||||||
val service2 = testKit2.spawn(pingPongBehavior)
|
|
||||||
system2.receptionist ! Register(PingKey, service2, regProbe2.ref)
|
|
||||||
regProbe2.expectMessage(Registered(PingKey, service2))
|
|
||||||
|
|
||||||
// then we join the cluster
|
|
||||||
val clusterNode2 = Cluster(system2)
|
|
||||||
clusterNode2.manager ! Join(clusterNode1.selfMember.address)
|
|
||||||
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up))
|
|
||||||
|
|
||||||
// and the subscriber on node1 should see the service
|
|
||||||
val remoteServiceRefs = regProbe1.expectMessageType[Listing].serviceInstances(PingKey)
|
|
||||||
val theRef = remoteServiceRefs.head
|
|
||||||
theRef ! Ping(regProbe1.ref)
|
|
||||||
regProbe1.expectMessage(Pong)
|
|
||||||
|
|
||||||
// abrupt termination
|
|
||||||
system2.terminate()
|
|
||||||
regProbe1.expectMessage(10.seconds, Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
|
||||||
} finally {
|
|
||||||
testKit1.shutdownTestKit()
|
|
||||||
testKit2.shutdownTestKit()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
"handle a new incarnation of the same node well" in {
|
|
||||||
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-4", ClusterReceptionistSpec.config)
|
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-4", ClusterReceptionistSpec.config)
|
||||||
val system1 = testKit1.system
|
val system1 = testKit1.system
|
||||||
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
||||||
|
|
@ -226,7 +220,107 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
val regProbe1 = TestProbe[Any]()(system1)
|
val regProbe1 = TestProbe[Any]()(system1)
|
||||||
val regProbe2 = TestProbe[Any]()(system2)
|
val regProbe2 = TestProbe[Any]()(system2)
|
||||||
|
|
||||||
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) == 2)
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2), 10.seconds)
|
||||||
|
|
||||||
|
system2.receptionist ! Subscribe(PingKey, regProbe2.ref)
|
||||||
|
regProbe2.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
||||||
|
val service1 = testKit1.spawn(pingPongBehavior)
|
||||||
|
system1.receptionist ! Register(PingKey, service1, regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Registered(PingKey, service1))
|
||||||
|
|
||||||
|
regProbe2.expectMessageType[Listing].serviceInstances(PingKey).size should ===(1)
|
||||||
|
|
||||||
|
val service2 = testKit2.spawn(pingPongBehavior)
|
||||||
|
system2.receptionist ! Register(PingKey, service2, regProbe2.ref)
|
||||||
|
regProbe2.expectMessage(Registered(PingKey, service2))
|
||||||
|
|
||||||
|
regProbe2.expectMessageType[Listing].serviceInstances(PingKey).size should ===(2)
|
||||||
|
|
||||||
|
akka.cluster.Cluster(system1.toUntyped).shutdown()
|
||||||
|
|
||||||
|
regProbe2.expectNoMessage(3.seconds)
|
||||||
|
|
||||||
|
clusterNode2.manager ! Down(clusterNode1.selfMember.address)
|
||||||
|
// service1 removed
|
||||||
|
regProbe2.expectMessage(10.seconds, Listing(PingKey, Set(service2)))
|
||||||
|
} finally {
|
||||||
|
testKit1.shutdownTestKit()
|
||||||
|
testKit2.shutdownTestKit()
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"work with services registered before node joins cluster" in {
|
||||||
|
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-5", ClusterReceptionistSpec.config)
|
||||||
|
val system1 = testKit1.system
|
||||||
|
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
||||||
|
val system2 = testKit2.system
|
||||||
|
try {
|
||||||
|
|
||||||
|
val clusterNode1 = Cluster(system1)
|
||||||
|
clusterNode1.manager ! Join(clusterNode1.selfMember.address)
|
||||||
|
|
||||||
|
val regProbe1 = TestProbe[Any]()(system1)
|
||||||
|
val regProbe2 = TestProbe[Any]()(system2)
|
||||||
|
|
||||||
|
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
||||||
|
val service2 = testKit2.spawn(pingPongBehavior)
|
||||||
|
system2.receptionist ! Register(PingKey, service2, regProbe2.ref)
|
||||||
|
regProbe2.expectMessage(Registered(PingKey, service2))
|
||||||
|
|
||||||
|
val reply2 = TestProbe[Listing]()(system2)
|
||||||
|
// awaitAssert because it is not immediately included in the registry (round trip to ddata)
|
||||||
|
reply2.awaitAssert {
|
||||||
|
system2.receptionist ! Find(PingKey, reply2.ref)
|
||||||
|
reply2.receiveMessage().serviceInstances(PingKey) should ===(Set(service2))
|
||||||
|
}
|
||||||
|
|
||||||
|
// and it shouldn't be removed (wait longer than pruning-interval)
|
||||||
|
Thread.sleep(2000)
|
||||||
|
system2.receptionist ! Find(PingKey, reply2.ref)
|
||||||
|
reply2.receiveMessage().serviceInstances(PingKey) should ===(Set(service2))
|
||||||
|
|
||||||
|
// then we join the cluster
|
||||||
|
val clusterNode2 = Cluster(system2)
|
||||||
|
clusterNode2.manager ! Join(clusterNode1.selfMember.address)
|
||||||
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2), 10.seconds)
|
||||||
|
|
||||||
|
// and the subscriber on node1 should see the service
|
||||||
|
val remoteServiceRefs = regProbe1.expectMessageType[Listing].serviceInstances(PingKey)
|
||||||
|
val theRef = remoteServiceRefs.head
|
||||||
|
theRef ! Ping(regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Pong)
|
||||||
|
|
||||||
|
// abrupt termination
|
||||||
|
Await.ready(system2.terminate(), 10.seconds)
|
||||||
|
clusterNode1.manager ! Down(clusterNode2.selfMember.address)
|
||||||
|
|
||||||
|
regProbe1.expectMessage(10.seconds, Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
} finally {
|
||||||
|
testKit1.shutdownTestKit()
|
||||||
|
testKit2.shutdownTestKit()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle a new incarnation of the same node well" in {
|
||||||
|
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-6", ClusterReceptionistSpec.config)
|
||||||
|
val system1 = testKit1.system
|
||||||
|
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
||||||
|
val system2 = testKit2.system
|
||||||
|
try {
|
||||||
|
|
||||||
|
val clusterNode1 = Cluster(system1)
|
||||||
|
clusterNode1.manager ! Join(clusterNode1.selfMember.address)
|
||||||
|
val clusterNode2 = Cluster(system2)
|
||||||
|
clusterNode2.manager ! Join(clusterNode1.selfMember.address)
|
||||||
|
|
||||||
|
val regProbe1 = TestProbe[Any]()(system1)
|
||||||
|
val regProbe2 = TestProbe[Any]()(system2)
|
||||||
|
|
||||||
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2), 10.seconds)
|
||||||
|
|
||||||
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
||||||
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
@ -241,42 +335,46 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
theRef ! Ping(regProbe1.ref)
|
theRef ! Ping(regProbe1.ref)
|
||||||
regProbe1.expectMessage(Pong)
|
regProbe1.expectMessage(Pong)
|
||||||
|
|
||||||
// FIXME do we need to blackhole the connection to system2 before terminating
|
|
||||||
// right now it doesn't work anyways though ;D
|
|
||||||
|
|
||||||
// abrupt termination but then a node with the same host:port comes online quickly
|
// abrupt termination but then a node with the same host:port comes online quickly
|
||||||
system1.log.debug("Terminating system2, uid: [{}]", clusterNode2.selfMember.uniqueAddress.longUid)
|
system1.log.debug("Terminating system2: [{}]", clusterNode2.selfMember.uniqueAddress)
|
||||||
Await.ready(system2.terminate(), 10.seconds)
|
Await.ready(system2.terminate(), 10.seconds)
|
||||||
|
|
||||||
val testKit3 = ActorTestKit(system1.name, testKit1.config)
|
val testKit3 = ActorTestKit(
|
||||||
|
system1.name,
|
||||||
|
ConfigFactory.parseString(s"""
|
||||||
|
akka.remote.netty.tcp.port = ${clusterNode2.selfMember.address.port.get}
|
||||||
|
akka.remote.artery.canonical.port = ${clusterNode2.selfMember.address.port.get}
|
||||||
|
# retry joining when existing member removed
|
||||||
|
akka.cluster.retry-unsuccessful-join-after = 1s
|
||||||
|
""").withFallback(config))
|
||||||
|
|
||||||
try {
|
try {
|
||||||
val system3 = testKit3.system
|
val system3 = testKit3.system
|
||||||
system1.log.debug(
|
|
||||||
"Starting system3 at same hostname port as system2, uid: [{}]",
|
|
||||||
Cluster(system3).selfMember.uniqueAddress.longUid)
|
|
||||||
val clusterNode3 = Cluster(system3)
|
val clusterNode3 = Cluster(system3)
|
||||||
clusterNode3.manager ! Join(clusterNode1.selfMember.address)
|
system1.log
|
||||||
|
.debug("Starting system3 at same hostname port as system2: [{}]", clusterNode3.selfMember.uniqueAddress)
|
||||||
|
// using JoinSeedNodes instead of Join to retry the join when existing member removed
|
||||||
|
clusterNode3.manager ! JoinSeedNodes(List(clusterNode1.selfMember.address))
|
||||||
val regProbe3 = TestProbe[Any]()(system3)
|
val regProbe3 = TestProbe[Any]()(system3)
|
||||||
|
|
||||||
// and registers the same service key
|
// and registers the same service key
|
||||||
val service3 = testKit3.spawn(pingPongBehavior, "instance")
|
val service3 = testKit3.spawn(pingPongBehavior, "instance")
|
||||||
system3.log.debug(
|
val service3Uid = service3.path.uid
|
||||||
"Spawning/registering ping service in new incarnation {}#{}",
|
system3.log.debug("Spawning/registering ping service in new incarnation {}", service3)
|
||||||
service3.path,
|
|
||||||
service3.path.uid)
|
|
||||||
system3.receptionist ! Register(PingKey, service3, regProbe3.ref)
|
system3.receptionist ! Register(PingKey, service3, regProbe3.ref)
|
||||||
regProbe3.expectMessage(Registered(PingKey, service3))
|
regProbe3.expectMessage(Registered(PingKey, service3))
|
||||||
system3.log.debug("Registered actor [{}#{}] for system3", service3.path, service3.path.uid)
|
system3.log.debug("Registered actor [{}] for system3", service3)
|
||||||
|
|
||||||
// make sure it joined fine and node1 has upped it
|
// make sure it joined fine and node1 has upped it
|
||||||
regProbe1.awaitAssert {
|
regProbe1.awaitAssert(
|
||||||
clusterNode1.state.members.exists(
|
{
|
||||||
m =>
|
clusterNode1.state.members.exists(
|
||||||
m.uniqueAddress == clusterNode3.selfMember.uniqueAddress &&
|
m =>
|
||||||
m.status == MemberStatus.Up &&
|
m.uniqueAddress == clusterNode3.selfMember.uniqueAddress &&
|
||||||
!clusterNode1.state.unreachable(m))
|
m.status == MemberStatus.Up &&
|
||||||
}
|
!clusterNode1.state.unreachable(m)) should ===(true)
|
||||||
|
},
|
||||||
|
10.seconds)
|
||||||
|
|
||||||
// we should get either empty message and then updated with the new incarnation actor
|
// we should get either empty message and then updated with the new incarnation actor
|
||||||
// or just updated with the new service directly
|
// or just updated with the new service directly
|
||||||
|
|
@ -289,6 +387,8 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
val ref = entries.head
|
val ref = entries.head
|
||||||
val service3RemotePath = RootActorPath(clusterNode3.selfMember.address) / "user" / "instance"
|
val service3RemotePath = RootActorPath(clusterNode3.selfMember.address) / "user" / "instance"
|
||||||
ref.path should ===(service3RemotePath)
|
ref.path should ===(service3RemotePath)
|
||||||
|
ref.path.uid should ===(service3Uid)
|
||||||
|
|
||||||
ref ! Ping(regProbe1.ref)
|
ref ! Ping(regProbe1.ref)
|
||||||
regProbe1.expectMessage(Pong)
|
regProbe1.expectMessage(Pong)
|
||||||
|
|
||||||
|
|
@ -301,13 +401,117 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// reproducer of issue #26284
|
||||||
|
"handle a new incarnation of the same node that is no longer part of same cluster" in {
|
||||||
|
val testKit1 = ActorTestKit(
|
||||||
|
"ClusterReceptionistSpec-test-7",
|
||||||
|
ConfigFactory.parseString("""
|
||||||
|
akka.cluster {
|
||||||
|
failure-detector.acceptable-heartbeat-pause = 20s
|
||||||
|
}
|
||||||
|
akka.cluster.typed.receptionist {
|
||||||
|
# it can be stressed more by using all
|
||||||
|
write-consistency = all
|
||||||
|
}
|
||||||
|
""").withFallback(ClusterReceptionistSpec.config))
|
||||||
|
val system1 = testKit1.system
|
||||||
|
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
||||||
|
val system2 = testKit2.system
|
||||||
|
try {
|
||||||
|
|
||||||
|
val clusterNode1 = Cluster(system1)
|
||||||
|
clusterNode1.manager ! Join(clusterNode1.selfMember.address)
|
||||||
|
val clusterNode2 = Cluster(system2)
|
||||||
|
clusterNode2.manager ! Join(clusterNode1.selfMember.address)
|
||||||
|
|
||||||
|
val regProbe1 = TestProbe[Any]()(system1)
|
||||||
|
val regProbe2 = TestProbe[Any]()(system2)
|
||||||
|
val reply1 = TestProbe[Listing]()(system1)
|
||||||
|
|
||||||
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2), 10.seconds)
|
||||||
|
|
||||||
|
system1.receptionist ! Subscribe(PingKey, regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
||||||
|
val service1 = testKit1.spawn(pingPongBehavior)
|
||||||
|
system1.receptionist ! Register(PingKey, service1, regProbe1.ref)
|
||||||
|
regProbe1.expectMessage(Registered(PingKey, service1))
|
||||||
|
regProbe1.expectMessage(Listing(PingKey, Set(service1)))
|
||||||
|
|
||||||
|
val service2 = testKit2.spawn(pingPongBehavior, "instance")
|
||||||
|
system2.receptionist ! Register(PingKey, service2, regProbe2.ref)
|
||||||
|
regProbe2.expectMessage(Registered(PingKey, service2))
|
||||||
|
|
||||||
|
// make sure we saw the first incarnation on node1
|
||||||
|
regProbe1.expectMessageType[Listing].serviceInstances(PingKey).size should ===(2)
|
||||||
|
|
||||||
|
// abrupt termination but then a node with the same host:port comes online quickly
|
||||||
|
system1.log.debug("Terminating system2: [{}]", clusterNode2.selfMember.uniqueAddress)
|
||||||
|
Await.ready(system2.terminate(), 10.seconds)
|
||||||
|
|
||||||
|
val testKit3 = ActorTestKit(
|
||||||
|
system1.name,
|
||||||
|
ConfigFactory.parseString(s"""
|
||||||
|
akka.remote.netty.tcp.port = ${clusterNode2.selfMember.address.port.get}
|
||||||
|
akka.remote.artery.canonical.port = ${clusterNode2.selfMember.address.port.get}
|
||||||
|
""").withFallback(config))
|
||||||
|
|
||||||
|
try {
|
||||||
|
val system3 = testKit3.system
|
||||||
|
val regProbe3 = TestProbe[Any]()(system3)
|
||||||
|
val clusterNode3 = Cluster(system3)
|
||||||
|
system1.log
|
||||||
|
.debug("Starting system3 at same hostname port as system2 [{}]", clusterNode3.selfMember.uniqueAddress)
|
||||||
|
// joining itself, i.e. not same cluster
|
||||||
|
clusterNode3.manager ! Join(clusterNode3.selfMember.address)
|
||||||
|
regProbe3.awaitAssert(clusterNode3.state.members.count(_.status == MemberStatus.Up) should ===(1))
|
||||||
|
|
||||||
|
// register another
|
||||||
|
Thread.sleep(2000)
|
||||||
|
val service1b = testKit1.spawn(pingPongBehavior)
|
||||||
|
system1.receptionist ! Register(PingKey, service1b, regProbe1.ref)
|
||||||
|
|
||||||
|
val service1c = testKit1.spawn(pingPongBehavior)
|
||||||
|
system1.receptionist ! Register(PingKey, service1c, regProbe1.ref)
|
||||||
|
|
||||||
|
system3.receptionist ! Subscribe(PingKey, regProbe3.ref)
|
||||||
|
// shouldn't get anything from the other cluster
|
||||||
|
regProbe3.expectMessage(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
||||||
|
// and registers the same service key
|
||||||
|
val service3 = testKit3.spawn(pingPongBehavior, "instance")
|
||||||
|
system3.log.debug("Spawning/registering ping service in new incarnation {}", service3)
|
||||||
|
system3.receptionist ! Register(PingKey, service3, regProbe3.ref)
|
||||||
|
regProbe3.expectMessage(Registered(PingKey, service3))
|
||||||
|
system3.log.debug("Registered actor [{}] for system3", service3)
|
||||||
|
|
||||||
|
// shouldn't get anything from the other cluster
|
||||||
|
regProbe3.expectMessage(Listing(PingKey, Set(service3)))
|
||||||
|
|
||||||
|
reply1.expectNoMessage(1.second)
|
||||||
|
system1.receptionist ! Find(PingKey, reply1.ref)
|
||||||
|
(reply1.receiveMessage().serviceInstances(PingKey) should contain).allOf(service1, service1b, service1c)
|
||||||
|
|
||||||
|
reply1.expectNoMessage(1.second)
|
||||||
|
system1.receptionist ! Find(PingKey, reply1.ref)
|
||||||
|
(reply1.receiveMessage().serviceInstances(PingKey) should contain).allOf(service1, service1b, service1c)
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
testKit3.shutdownTestKit()
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
testKit1.shutdownTestKit()
|
||||||
|
testKit2.shutdownTestKit()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
"not lose removals on concurrent updates to same key" in {
|
"not lose removals on concurrent updates to same key" in {
|
||||||
val config = ConfigFactory.parseString("""
|
val config = ConfigFactory.parseString("""
|
||||||
# disable delta propagation so we can have repeatable concurrent writes
|
# disable delta propagation so we can have repeatable concurrent writes
|
||||||
# without delta reaching between nodes already
|
# without delta reaching between nodes already
|
||||||
akka.cluster.distributed-data.delta-crdt.enabled=false
|
akka.cluster.distributed-data.delta-crdt.enabled=false
|
||||||
""").withFallback(ClusterReceptionistSpec.config)
|
""").withFallback(ClusterReceptionistSpec.config)
|
||||||
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-5", config)
|
val testKit1 = ActorTestKit("ClusterReceptionistSpec-test-8", config)
|
||||||
val system1 = testKit1.system
|
val system1 = testKit1.system
|
||||||
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
val testKit2 = ActorTestKit(system1.name, system1.settings.config)
|
||||||
val system2 = testKit2.system
|
val system2 = testKit2.system
|
||||||
|
|
@ -322,7 +526,7 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
|
||||||
val regProbe1 = TestProbe[AnyRef]()(system1)
|
val regProbe1 = TestProbe[AnyRef]()(system1)
|
||||||
val regProbe2 = TestProbe[AnyRef]()(system2)
|
val regProbe2 = TestProbe[AnyRef]()(system2)
|
||||||
|
|
||||||
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) == 2)
|
regProbe1.awaitAssert(clusterNode1.state.members.count(_.status == MemberStatus.Up) should ===(2))
|
||||||
|
|
||||||
// one actor on each node up front
|
// one actor on each node up front
|
||||||
val actor1 = testKit1.spawn(Behaviors.receive[AnyRef] {
|
val actor1 = testKit1.spawn(Behaviors.receive[AnyRef] {
|
||||||
|
|
|
||||||
|
|
@ -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!
|
// 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[IncompatibleMethTypeProblem]("akka.cluster.ddata.ORSet.clear")
|
||||||
ProblemFilters.exclude[DirectAbstractMethodProblem]("akka.cluster.ddata.PruningState.addSeen")
|
ProblemFilters.exclude[DirectAbstractMethodProblem]("akka.cluster.ddata.PruningState.addSeen")
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("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 OtherMessage key = 1;
|
||||||
required sint32 consistency = 2;
|
required sint32 consistency = 2;
|
||||||
required uint32 timeout = 3;
|
required uint32 timeout = 3;
|
||||||
optional OtherMessage request = 4;
|
optional OtherMessage request = 4;
|
||||||
}
|
}
|
||||||
|
|
||||||
message GetSuccess {
|
message GetSuccess {
|
||||||
|
|
@ -48,6 +48,7 @@ message Changed {
|
||||||
message Write {
|
message Write {
|
||||||
required string key = 1;
|
required string key = 1;
|
||||||
required DataEnvelope envelope = 2;
|
required DataEnvelope envelope = 2;
|
||||||
|
optional UniqueAddress fromNode = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
// message WriteAck, via Empty
|
// message WriteAck, via Empty
|
||||||
|
|
@ -57,6 +58,7 @@ message Empty {
|
||||||
|
|
||||||
message Read {
|
message Read {
|
||||||
required string key = 1;
|
required string key = 1;
|
||||||
|
optional UniqueAddress fromNode = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
message ReadResult {
|
message ReadResult {
|
||||||
|
|
@ -86,6 +88,8 @@ message Status {
|
||||||
required uint32 chunk = 1;
|
required uint32 chunk = 1;
|
||||||
required uint32 totChunks = 2;
|
required uint32 totChunks = 2;
|
||||||
repeated Entry entries = 3;
|
repeated Entry entries = 3;
|
||||||
|
optional sfixed64 toSystemUid = 4;
|
||||||
|
optional sfixed64 fromSystemUid = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
message Gossip {
|
message Gossip {
|
||||||
|
|
@ -96,6 +100,8 @@ message Gossip {
|
||||||
|
|
||||||
required bool sendBack = 1;
|
required bool sendBack = 1;
|
||||||
repeated Entry entries = 2;
|
repeated Entry entries = 2;
|
||||||
|
optional sfixed64 toSystemUid = 3;
|
||||||
|
optional sfixed64 fromSystemUid = 4;
|
||||||
}
|
}
|
||||||
|
|
||||||
message DeltaPropagation {
|
message DeltaPropagation {
|
||||||
|
|
@ -108,7 +114,7 @@ message DeltaPropagation {
|
||||||
|
|
||||||
required UniqueAddress fromNode = 1;
|
required UniqueAddress fromNode = 1;
|
||||||
repeated Entry entries = 2;
|
repeated Entry entries = 2;
|
||||||
optional bool reply = 3; // no reply if not set
|
optional bool reply = 3; // no reply if not set
|
||||||
}
|
}
|
||||||
|
|
||||||
message UniqueAddress {
|
message UniqueAddress {
|
||||||
|
|
|
||||||
|
|
@ -6,8 +6,8 @@ package akka.cluster.ddata
|
||||||
|
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
|
|
||||||
import akka.actor.Address
|
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.cluster.UniqueAddress
|
||||||
import akka.cluster.ddata.Key.KeyId
|
import akka.cluster.ddata.Key.KeyId
|
||||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
||||||
|
|
@ -25,12 +25,12 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
def propagationCount: Long = _propagationCount
|
def propagationCount: Long = _propagationCount
|
||||||
private var deltaCounter = Map.empty[KeyId, Long]
|
private var deltaCounter = Map.empty[KeyId, Long]
|
||||||
private var deltaEntries = Map.empty[KeyId, TreeMap[Long, ReplicatedData]]
|
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
|
private var deltaNodeRoundRobinCounter = 0L
|
||||||
|
|
||||||
def gossipIntervalDivisor: Int
|
def gossipIntervalDivisor: Int
|
||||||
|
|
||||||
def allNodes: Vector[Address]
|
def allNodes: Vector[UniqueAddress]
|
||||||
|
|
||||||
def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation
|
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))
|
math.min(math.max((allNodesSize / gossipIntervalDivisor) + 1, 2), math.min(allNodesSize, 10))
|
||||||
}
|
}
|
||||||
|
|
||||||
def collectPropagations(): Map[Address, DeltaPropagation] = {
|
def collectPropagations(): Map[UniqueAddress, DeltaPropagation] = {
|
||||||
_propagationCount += 1
|
_propagationCount += 1
|
||||||
val all = allNodes
|
val all = allNodes
|
||||||
if (all.isEmpty)
|
if (all.isEmpty)
|
||||||
|
|
@ -90,7 +90,7 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
}
|
}
|
||||||
deltaNodeRoundRobinCounter += sliceSize
|
deltaNodeRoundRobinCounter += sliceSize
|
||||||
|
|
||||||
var result = Map.empty[Address, DeltaPropagation]
|
var result = Map.empty[UniqueAddress, DeltaPropagation]
|
||||||
|
|
||||||
var cache = Map.empty[(KeyId, Long, Long), ReplicatedData]
|
var cache = Map.empty[(KeyId, Long, Long), ReplicatedData]
|
||||||
slice.foreach { node =>
|
slice.foreach { node =>
|
||||||
|
|
@ -99,7 +99,7 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
var deltas = Map.empty[KeyId, (ReplicatedData, Long, Long)]
|
var deltas = Map.empty[KeyId, (ReplicatedData, Long, Long)]
|
||||||
deltaEntries.foreach {
|
deltaEntries.foreach {
|
||||||
case (key, entries) =>
|
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 j = deltaSentToNodeForKey.getOrElse(node, 0L)
|
||||||
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
|
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
|
||||||
if (deltaEntriesAfterJ.nonEmpty) {
|
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 {
|
deltaSentToNode.get(key) match {
|
||||||
case None => 0L
|
case None => 0L
|
||||||
case Some(deltaSentToNodeForKey) =>
|
case Some(deltaSentToNodeForKey) =>
|
||||||
|
|
@ -188,7 +188,7 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def cleanupRemovedNode(address: Address): Unit = {
|
def cleanupRemovedNode(address: UniqueAddress): Unit = {
|
||||||
deltaSentToNode = deltaSentToNode.map {
|
deltaSentToNode = deltaSentToNode.map {
|
||||||
case (key, deltaSentToNodeForKey) =>
|
case (key, deltaSentToNodeForKey) =>
|
||||||
key -> (deltaSentToNodeForKey - address)
|
key -> (deltaSentToNodeForKey - address)
|
||||||
|
|
|
||||||
|
|
@ -5,15 +5,18 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import java.security.MessageDigest
|
import java.security.MessageDigest
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.duration.FiniteDuration
|
import scala.concurrent.duration.FiniteDuration
|
||||||
import java.util.concurrent.ThreadLocalRandom
|
import java.util.concurrent.ThreadLocalRandom
|
||||||
|
|
||||||
import scala.util.Failure
|
import scala.util.Failure
|
||||||
import scala.util.Success
|
import scala.util.Success
|
||||||
import scala.util.Try
|
import scala.util.Try
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.actor.ActorLogging
|
import akka.actor.ActorLogging
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
|
|
@ -34,24 +37,31 @@ import akka.serialization.SerializationExtension
|
||||||
import akka.util.ByteString
|
import akka.util.ByteString
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import java.util.function.{ Function => JFunction }
|
import java.util.function.{ Function => JFunction }
|
||||||
|
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
import akka.actor.DeadLetterSuppression
|
import akka.actor.DeadLetterSuppression
|
||||||
import akka.cluster.ddata.Key.KeyR
|
import akka.cluster.ddata.Key.KeyR
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
|
|
||||||
import akka.cluster.ddata.DurableStore._
|
import akka.cluster.ddata.DurableStore._
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.SupervisorStrategy
|
import akka.actor.SupervisorStrategy
|
||||||
import akka.actor.OneForOneStrategy
|
import akka.actor.OneForOneStrategy
|
||||||
import akka.actor.ActorInitializationException
|
import akka.actor.ActorInitializationException
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
|
|
||||||
import akka.util.Helpers.toRootLowerCase
|
import akka.util.Helpers.toRootLowerCase
|
||||||
import akka.actor.Cancellable
|
import akka.actor.Cancellable
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
|
|
||||||
import akka.cluster.ddata.Key.KeyId
|
import akka.cluster.ddata.Key.KeyId
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import scala.collection.immutable.TreeSet
|
import scala.collection.immutable.TreeSet
|
||||||
|
|
||||||
import akka.cluster.MemberStatus
|
import akka.cluster.MemberStatus
|
||||||
import scala.annotation.varargs
|
import scala.annotation.varargs
|
||||||
|
|
||||||
|
import akka.event.Logging
|
||||||
import akka.util.JavaDurationConverters._
|
import akka.util.JavaDurationConverters._
|
||||||
import akka.util.ccompat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
|
|
@ -790,10 +800,20 @@ object Replicator {
|
||||||
case object DeltaPropagationTick
|
case object DeltaPropagationTick
|
||||||
case object RemovedNodePruningTick
|
case object RemovedNodePruningTick
|
||||||
case object ClockTick
|
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 WriteAck extends ReplicatorMessage with DeadLetterSuppression
|
||||||
case object WriteNack 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 ReadResult(envelope: Option[DataEnvelope]) extends ReplicatorMessage with DeadLetterSuppression
|
||||||
final case class ReadRepair(key: KeyId, envelope: DataEnvelope)
|
final case class ReadRepair(key: KeyId, envelope: DataEnvelope)
|
||||||
case object ReadRepairAck
|
case object ReadRepairAck
|
||||||
|
|
@ -946,7 +966,14 @@ object Replicator {
|
||||||
override def merge(that: ReplicatedData): ReplicatedData = DeletedData
|
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 =
|
override def toString: String =
|
||||||
(digests
|
(digests
|
||||||
.map {
|
.map {
|
||||||
|
|
@ -954,11 +981,22 @@ object Replicator {
|
||||||
})
|
})
|
||||||
.mkString("Status(", ", ", ")")
|
.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 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
|
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 {
|
object DeltaPropagation {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -1186,6 +1224,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val cluster = Cluster(context.system)
|
val cluster = Cluster(context.system)
|
||||||
val selfAddress = cluster.selfAddress
|
val selfAddress = cluster.selfAddress
|
||||||
val selfUniqueAddress = cluster.selfUniqueAddress
|
val selfUniqueAddress = cluster.selfUniqueAddress
|
||||||
|
val selfFromSystemUid = Some(selfUniqueAddress.longUid)
|
||||||
|
|
||||||
require(!cluster.isTerminated, "Cluster node must not be terminated")
|
require(!cluster.isTerminated, "Cluster node must not be terminated")
|
||||||
require(
|
require(
|
||||||
|
|
@ -1223,9 +1262,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
|
|
||||||
val deltaPropagationSelector = new DeltaPropagationSelector {
|
val deltaPropagationSelector = new DeltaPropagationSelector {
|
||||||
override val gossipIntervalDivisor = 5
|
override val gossipIntervalDivisor = 5
|
||||||
override def allNodes: Vector[Address] = {
|
override def allNodes: Vector[UniqueAddress] = {
|
||||||
// TODO optimize, by maintaining a sorted instance variable instead
|
// 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
|
override def maxDeltaSize: Int = settings.maxDeltaSize
|
||||||
|
|
@ -1256,11 +1295,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
.schedule(deltaPropagationInterval, deltaPropagationInterval, self, DeltaPropagationTick))
|
.schedule(deltaPropagationInterval, deltaPropagationInterval, self, DeltaPropagationTick))
|
||||||
} else None
|
} else None
|
||||||
|
|
||||||
// cluster nodes, doesn't contain selfAddress
|
// cluster nodes, doesn't contain selfAddress, doesn't contain joining and weaklyUp
|
||||||
var nodes: Set[Address] = Set.empty
|
var nodes: Set[UniqueAddress] = Set.empty
|
||||||
|
|
||||||
// cluster weaklyUp nodes, doesn't contain selfAddress
|
// 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
|
var removedNodes: Map[UniqueAddress, Long] = Map.empty
|
||||||
// all nodes sorted with the leader first
|
// 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
|
// for pruning timeouts are based on clock that is only increased when all nodes are reachable
|
||||||
var previousClockTime = System.nanoTime()
|
var previousClockTime = System.nanoTime()
|
||||||
var allReachableClockTime = 0L
|
var allReachableClockTime = 0L
|
||||||
var unreachable = Set.empty[Address]
|
var unreachable = Set.empty[UniqueAddress]
|
||||||
|
|
||||||
// the actual data
|
// the actual data
|
||||||
var dataEntries = Map.empty[KeyId, (DataEnvelope, Digest)]
|
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
|
// MUST use replyTo instead of sender() and forward from normalReceive, because of the stash in load
|
||||||
val normalReceive: Receive = {
|
val normalReceive: Receive = {
|
||||||
case Get(key, consistency, req) => receiveGet(key, consistency, req)
|
case msg: DestinationSystemUid =>
|
||||||
case u @ Update(key, writeC, req) => receiveUpdate(key, u.modify, writeC, req)
|
msg.toSystemUid match {
|
||||||
case Read(key) => receiveRead(key)
|
case Some(uid) if uid != selfUniqueAddress.longUid =>
|
||||||
case Write(key, envelope) => receiveWrite(key, envelope)
|
// When restarting a node with same host:port it is possible that a Replicator on another node
|
||||||
case ReadRepair(key, envelope) => receiveReadRepair(key, envelope)
|
// is sending messages to the restarted node even if it hasn't joined the same cluster.
|
||||||
case DeltaPropagation(from, reply, deltas) => receiveDeltaPropagation(from, reply, deltas)
|
// Therefore we check that the message was intended for this incarnation and otherwise
|
||||||
case FlushChanges => receiveFlushChanges()
|
// it is discarded.
|
||||||
case DeltaPropagationTick => receiveDeltaPropagationTick()
|
log.info(
|
||||||
case GossipTick => receiveGossipTick()
|
"Ignoring message [{}] from [{}] intended for system uid [{}], self uid is [{}]",
|
||||||
case ClockTick => receiveClockTick()
|
Logging.simpleName(msg),
|
||||||
case Status(otherDigests, chunk, totChunks) => receiveStatus(otherDigests, chunk, totChunks)
|
replyTo,
|
||||||
case Gossip(updatedData, sendBack) => receiveGossip(updatedData, sendBack)
|
uid,
|
||||||
case Subscribe(key, subscriber) => receiveSubscribe(key, subscriber)
|
selfUniqueAddress.longUid)
|
||||||
case Unsubscribe(key, subscriber) => receiveUnsubscribe(key, subscriber)
|
case _ =>
|
||||||
case Terminated(ref) => receiveTerminated(ref)
|
msg match {
|
||||||
case MemberWeaklyUp(m) => receiveWeaklyUpMemberUp(m)
|
case Status(otherDigests, chunk, totChunks, _, fromSystemUid) =>
|
||||||
case MemberUp(m) => receiveMemberUp(m)
|
receiveStatus(otherDigests, chunk, totChunks, fromSystemUid)
|
||||||
case MemberRemoved(m, _) => receiveMemberRemoved(m)
|
case Gossip(updatedData, sendBack, _, fromSystemUid) =>
|
||||||
case evt: MemberEvent => receiveOtherMemberEvent(evt.member)
|
receiveGossip(updatedData, sendBack, fromSystemUid)
|
||||||
case UnreachableMember(m) => receiveUnreachable(m)
|
}
|
||||||
case ReachableMember(m) => receiveReachable(m)
|
}
|
||||||
case GetKeyIds => receiveGetKeyIds()
|
|
||||||
case Delete(key, consistency, req) => receiveDelete(key, consistency, req)
|
case msg: SendingSystemUid =>
|
||||||
case RemovedNodePruningTick => receiveRemovedNodePruningTick()
|
msg.fromNode match {
|
||||||
case GetReplicaCount => receiveGetReplicaCount()
|
case Some(fromNode) if !isKnownNode(fromNode) =>
|
||||||
case TestFullStateGossip(enabled) => fullStateGossipEnabled = enabled
|
// 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 = {
|
def receiveGet(key: KeyR, consistency: ReadConsistency, req: Option[Any]): Unit = {
|
||||||
|
|
@ -1438,7 +1520,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
} else
|
} else
|
||||||
context.actorOf(
|
context.actorOf(
|
||||||
ReadAggregator
|
ReadAggregator
|
||||||
.props(key, consistency, req, nodes, unreachable, localValue, replyTo)
|
.props(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo)
|
||||||
.withDispatcher(context.props.dispatcher))
|
.withDispatcher(context.props.dispatcher))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1521,7 +1603,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val writeAggregator =
|
val writeAggregator =
|
||||||
context.actorOf(
|
context.actorOf(
|
||||||
WriteAggregator
|
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))
|
.withDispatcher(context.props.dispatcher))
|
||||||
if (durable) {
|
if (durable) {
|
||||||
durableStore ! Store(
|
durableStore ! Store(
|
||||||
|
|
@ -1630,7 +1722,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val writeAggregator =
|
val writeAggregator =
|
||||||
context.actorOf(
|
context.actorOf(
|
||||||
WriteAggregator
|
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))
|
.withDispatcher(context.props.dispatcher))
|
||||||
if (durable) {
|
if (durable) {
|
||||||
durableStore ! Store(
|
durableStore ! Store(
|
||||||
|
|
@ -1811,13 +1913,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
|
|
||||||
def receiveGossipTick(): Unit = {
|
def receiveGossipTick(): Unit = {
|
||||||
if (fullStateGossipEnabled)
|
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 to = replica(address)
|
||||||
|
val toSystemUid = Some(address.longUid)
|
||||||
if (dataEntries.size <= maxDeltaElements) {
|
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
|
to ! status
|
||||||
} else {
|
} else {
|
||||||
val totChunks = dataEntries.size / maxDeltaElements
|
val totChunks = dataEntries.size / maxDeltaElements
|
||||||
|
|
@ -1831,19 +1939,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val chunk = (statusCount % totChunks).toInt
|
val chunk = (statusCount % totChunks).toInt
|
||||||
val status = Status(dataEntries.collect {
|
val status = Status(dataEntries.collect {
|
||||||
case (key, (_, _)) if math.abs(key.hashCode % totChunks) == chunk => (key, getDigest(key))
|
case (key, (_, _)) if math.abs(key.hashCode % totChunks) == chunk => (key, getDigest(key))
|
||||||
}, chunk, totChunks)
|
}, chunk, totChunks, toSystemUid, selfFromSystemUid)
|
||||||
to ! status
|
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)))
|
if (addresses.isEmpty) None else Some(addresses(ThreadLocalRandom.current.nextInt(addresses.size)))
|
||||||
|
|
||||||
def replica(address: Address): ActorSelection =
|
def replica(node: UniqueAddress): ActorSelection =
|
||||||
context.actorSelection(self.path.toStringWithAddress(address))
|
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)
|
if (log.isDebugEnabled)
|
||||||
log.debug(
|
log.debug(
|
||||||
"Received gossip status from [{}], chunk [{}] of [{}] containing [{}]",
|
"Received gossip status from [{}], chunk [{}] of [{}] containing [{}]",
|
||||||
|
|
@ -1868,7 +1976,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
if (keys.nonEmpty) {
|
if (keys.nonEmpty) {
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
|
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
|
replyTo ! g
|
||||||
}
|
}
|
||||||
val myMissingKeys = otherKeys.diff(myKeys)
|
val myMissingKeys = otherKeys.diff(myKeys)
|
||||||
|
|
@ -1878,12 +1990,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
"Sending gossip status to [{}], requesting missing [{}]",
|
"Sending gossip status to [{}], requesting missing [{}]",
|
||||||
replyTo.path.address,
|
replyTo.path.address,
|
||||||
myMissingKeys.mkString(", "))
|
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
|
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)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
|
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
|
||||||
var replyData = Map.empty[KeyId, DataEnvelope]
|
var replyData = Map.empty[KeyId, DataEnvelope]
|
||||||
|
|
@ -1899,7 +2016,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (sendBack && replyData.nonEmpty)
|
if (sendBack && replyData.nonEmpty)
|
||||||
replyTo ! Gossip(replyData, sendBack = false)
|
replyTo ! Gossip(replyData, sendBack = false, fromSystemUid, selfFromSystemUid)
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveSubscribe(key: KeyR, subscriber: ActorRef): Unit = {
|
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)
|
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 =
|
def receiveMemberUp(m: Member): Unit =
|
||||||
if (matchingRole(m)) {
|
if (matchingRole(m)) {
|
||||||
leader += m
|
leader += m
|
||||||
if (m.address != selfAddress) {
|
if (m.address != selfAddress) {
|
||||||
nodes += m.address
|
nodes += m.uniqueAddress
|
||||||
weaklyUpNodes -= m.address
|
weaklyUpNodes -= m.uniqueAddress
|
||||||
|
joiningNodes -= m.uniqueAddress
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1960,14 +2084,15 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
if (m.address == selfAddress)
|
if (m.address == selfAddress)
|
||||||
context.stop(self)
|
context.stop(self)
|
||||||
else if (matchingRole(m)) {
|
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
|
// filter, it's possible that the ordering is changed since it based on MemberStatus
|
||||||
leader = leader.filterNot(_.uniqueAddress == m.uniqueAddress)
|
leader = leader.filterNot(_.uniqueAddress == m.uniqueAddress)
|
||||||
nodes -= m.address
|
nodes -= m.uniqueAddress
|
||||||
weaklyUpNodes -= m.address
|
weaklyUpNodes -= m.uniqueAddress
|
||||||
log.debug("adding removed node [{}] from MemberRemoved", m.uniqueAddress)
|
joiningNodes -= m.uniqueAddress
|
||||||
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
|
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
|
||||||
unreachable -= m.address
|
unreachable -= m.uniqueAddress
|
||||||
deltaPropagationSelector.cleanupRemovedNode(m.address)
|
deltaPropagationSelector.cleanupRemovedNode(m.uniqueAddress)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1979,10 +2104,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveUnreachable(m: Member): Unit =
|
def receiveUnreachable(m: Member): Unit =
|
||||||
if (matchingRole(m)) unreachable += m.address
|
if (matchingRole(m)) unreachable += m.uniqueAddress
|
||||||
|
|
||||||
def receiveReachable(m: Member): Unit =
|
def receiveReachable(m: Member): Unit =
|
||||||
if (matchingRole(m)) unreachable -= m.address
|
if (matchingRole(m)) unreachable -= m.uniqueAddress
|
||||||
|
|
||||||
def receiveClockTick(): Unit = {
|
def receiveClockTick(): Unit = {
|
||||||
val now = System.nanoTime()
|
val now = System.nanoTime()
|
||||||
|
|
@ -2004,11 +2129,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
|
|
||||||
def collectRemovedNodes(): Unit = {
|
def collectRemovedNodes(): Unit = {
|
||||||
val knownNodes = nodes.union(weaklyUpNodes).union(removedNodes.keySet.map(_.address))
|
val knownNodes = allNodes.union(removedNodes.keySet)
|
||||||
val newRemovedNodes =
|
val newRemovedNodes =
|
||||||
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
|
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
|
||||||
case (acc, (_, (DataEnvelope(data: RemovedNodePruning, _, _), _))) =>
|
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, _) =>
|
case (acc, _) =>
|
||||||
acc
|
acc
|
||||||
}
|
}
|
||||||
|
|
@ -2023,7 +2148,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
// initiate pruning for removed nodes
|
// initiate pruning for removed nodes
|
||||||
val removedSet: Set[UniqueAddress] = removedNodes.iterator
|
val removedSet: Set[UniqueAddress] = removedNodes.iterator
|
||||||
.collect {
|
.collect {
|
||||||
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) => r
|
case (r, t) if (allReachableClockTime - t) > maxPruningDisseminationNanos => r
|
||||||
}
|
}
|
||||||
.to(immutable.Set)
|
.to(immutable.Set)
|
||||||
|
|
||||||
|
|
@ -2053,7 +2178,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
|
|
||||||
def performRemovedNodePruning(): Unit = {
|
def performRemovedNodePruning(): Unit = {
|
||||||
// perform pruning when all seen Init
|
// perform pruning when all seen Init
|
||||||
val allNodes = nodes.union(weaklyUpNodes)
|
val all = allNodes
|
||||||
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
|
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
|
||||||
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
|
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
|
||||||
dataEntries.foreach {
|
dataEntries.foreach {
|
||||||
|
|
@ -2061,7 +2186,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
pruning.foreach {
|
pruning.foreach {
|
||||||
case (removed, PruningInitialized(owner, seen))
|
case (removed, PruningInitialized(owner, seen))
|
||||||
if owner == selfUniqueAddress
|
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)
|
val newEnvelope = envelope.prune(removed, if (isDurable(key)) durablePruningPerformed else pruningPerformed)
|
||||||
log.debug("Perform pruning of [{}] from [{}] to [{}]", key, removed, selfUniqueAddress)
|
log.debug("Perform pruning of [{}] from [{}] to [{}]", key, removed, selfUniqueAddress)
|
||||||
setData(key, newEnvelope)
|
setData(key, newEnvelope)
|
||||||
|
|
@ -2124,22 +2249,23 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
import ReadWriteAggregator._
|
import ReadWriteAggregator._
|
||||||
|
|
||||||
def timeout: FiniteDuration
|
def timeout: FiniteDuration
|
||||||
def nodes: Set[Address]
|
def nodes: Set[UniqueAddress]
|
||||||
def unreachable: Set[Address]
|
def unreachable: Set[UniqueAddress]
|
||||||
def reachableNodes: Set[Address] = nodes.diff(unreachable)
|
def reachableNodes: Set[UniqueAddress] = nodes.diff(unreachable)
|
||||||
|
|
||||||
import context.dispatcher
|
import context.dispatcher
|
||||||
var sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
|
var sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
|
||||||
var timeoutSchedule = context.system.scheduler.scheduleOnce(timeout, self, ReceiveTimeout)
|
var timeoutSchedule = context.system.scheduler.scheduleOnce(timeout, self, ReceiveTimeout)
|
||||||
|
|
||||||
var remaining = nodes
|
var remaining = nodes.map(_.address)
|
||||||
|
|
||||||
def doneWhenRemainingSize: Int
|
def doneWhenRemainingSize: Int
|
||||||
|
|
||||||
def primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas: Boolean): (Vector[Address], Vector[Address]) = {
|
def primaryAndSecondaryNodes(
|
||||||
|
requiresCausalDeliveryOfDeltas: Boolean): (Vector[UniqueAddress], Vector[UniqueAddress]) = {
|
||||||
val primarySize = nodes.size - doneWhenRemainingSize
|
val primarySize = nodes.size - doneWhenRemainingSize
|
||||||
if (primarySize >= nodes.size)
|
if (primarySize >= nodes.size)
|
||||||
(nodes.toVector, Vector.empty[Address])
|
(nodes.toVector, Vector.empty[UniqueAddress])
|
||||||
else {
|
else {
|
||||||
// Prefer to use reachable nodes over the unreachable nodes first.
|
// Prefer to use reachable nodes over the unreachable nodes first.
|
||||||
// When RequiresCausalDeliveryOfDeltas use deterministic order to so that sequence numbers of subsequent
|
// 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()
|
timeoutSchedule.cancel()
|
||||||
}
|
}
|
||||||
|
|
||||||
def replica(address: Address): ActorSelection =
|
def replica(node: UniqueAddress): ActorSelection =
|
||||||
context.actorSelection(context.parent.path.toStringWithAddress(address))
|
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],
|
delta: Option[Replicator.Internal.Delta],
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
unreachable: Set[Address],
|
nodes: Set[UniqueAddress],
|
||||||
|
unreachable: Set[UniqueAddress],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean): Props =
|
durable: Boolean): Props =
|
||||||
Props(new WriteAggregator(key, envelope, delta, consistency, req, nodes, unreachable, replyTo, durable))
|
Props(
|
||||||
.withDeploy(Deploy.local)
|
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],
|
delta: Option[Replicator.Internal.Delta],
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
override val nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
override val unreachable: Set[Address],
|
override val nodes: Set[UniqueAddress],
|
||||||
|
override val unreachable: Set[UniqueAddress],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean)
|
durable: Boolean)
|
||||||
extends ReadWriteAggregator {
|
extends ReadWriteAggregator
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
import Replicator._
|
import Replicator._
|
||||||
import Replicator.Internal._
|
import Replicator.Internal._
|
||||||
import ReadWriteAggregator._
|
import ReadWriteAggregator._
|
||||||
|
|
||||||
val selfUniqueAddress = Cluster(context.system).selfUniqueAddress
|
|
||||||
|
|
||||||
override def timeout: FiniteDuration = consistency.timeout
|
override def timeout: FiniteDuration = consistency.timeout
|
||||||
|
|
||||||
override val doneWhenRemainingSize = consistency match {
|
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")
|
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 {
|
val deltaMsg = delta match {
|
||||||
case None => None
|
case None => None
|
||||||
case Some(d) => Some(DeltaPropagation(selfUniqueAddress, reply = true, Map(key.id -> d)))
|
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
|
// 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.
|
// 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.
|
// 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 }
|
secondaryNodes.foreach { replica(_) ! writeMsg }
|
||||||
case ReceiveTimeout =>
|
case ReceiveTimeout =>
|
||||||
|
|
@ -2309,11 +2449,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
consistency: Replicator.ReadConsistency,
|
consistency: Replicator.ReadConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
unreachable: Set[Address],
|
nodes: Set[UniqueAddress],
|
||||||
|
unreachable: Set[UniqueAddress],
|
||||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||||
replyTo: ActorRef): Props =
|
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,
|
key: KeyR,
|
||||||
consistency: Replicator.ReadConsistency,
|
consistency: Replicator.ReadConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
override val nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
override val unreachable: Set[Address],
|
override val nodes: Set[UniqueAddress],
|
||||||
|
override val unreachable: Set[UniqueAddress],
|
||||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||||
replyTo: ActorRef)
|
replyTo: ActorRef)
|
||||||
extends ReadWriteAggregator {
|
extends ReadWriteAggregator {
|
||||||
|
|
@ -2348,7 +2491,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
throw new IllegalArgumentException("ReadLocal not supported by ReadAggregator")
|
throw new IllegalArgumentException("ReadLocal not supported by ReadAggregator")
|
||||||
}
|
}
|
||||||
|
|
||||||
val readMsg = Read(key.id)
|
val readMsg = Read(key.id, Some(selfUniqueAddress))
|
||||||
|
|
||||||
private val (primaryNodes, secondaryNodes) = {
|
private val (primaryNodes, secondaryNodes) = {
|
||||||
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas = false)
|
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas = false)
|
||||||
|
|
|
||||||
|
|
@ -50,14 +50,14 @@ private object ReplicatedDataSerializer {
|
||||||
@silent
|
@silent
|
||||||
private final def compareKeys(t1: Any, t2: Any): Int = (t1, t2) match {
|
private final def compareKeys(t1: Any, t2: Any): Int = (t1, t2) match {
|
||||||
case (k1: String, k2: String) => k1.compareTo(k2)
|
case (k1: String, k2: String) => k1.compareTo(k2)
|
||||||
case (k1: String, k2) => -1
|
case (_: String, _) => -1
|
||||||
case (k1, k2: String) => 1
|
case (_, _: String) => 1
|
||||||
case (k1: Int, k2: Int) => k1.compareTo(k2)
|
case (k1: Int, k2: Int) => k1.compareTo(k2)
|
||||||
case (k1: Int, k2) => -1
|
case (_: Int, _) => -1
|
||||||
case (k1, k2: Int) => 1
|
case (_, _: Int) => 1
|
||||||
case (k1: Long, k2: Long) => k1.compareTo(k2)
|
case (k1: Long, k2: Long) => k1.compareTo(k2)
|
||||||
case (k1: Long, k2) => -1
|
case (_: Long, _) => -1
|
||||||
case (k1, k2: Long) => 1
|
case (_, _: Long) => 1
|
||||||
case (k1: OtherMessage, k2: OtherMessage) => OtherMessageComparator.compare(k1, k2)
|
case (k1: OtherMessage, k2: OtherMessage) => OtherMessageComparator.compare(k1, k2)
|
||||||
case (k1, k2) =>
|
case (k1, k2) =>
|
||||||
throw new IllegalStateException(
|
throw new IllegalStateException(
|
||||||
|
|
|
||||||
|
|
@ -267,15 +267,21 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
case (key, digest) =>
|
case (key, digest) =>
|
||||||
b.addEntries(dm.Status.Entry.newBuilder().setKey(key).setDigest(ByteString.copyFrom(digest.toArray)))
|
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()
|
b.build()
|
||||||
}
|
}
|
||||||
|
|
||||||
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
||||||
val status = dm.Status.parseFrom(bytes)
|
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(
|
||||||
status.getEntriesList.asScala.iterator.map(e => e.getKey -> AkkaByteString(e.getDigest.toByteArray())).toMap,
|
status.getEntriesList.asScala.iterator.map(e => e.getKey -> AkkaByteString(e.getDigest.toByteArray())).toMap,
|
||||||
status.getChunk,
|
status.getChunk,
|
||||||
status.getTotChunks)
|
status.getTotChunks,
|
||||||
|
toSystemUid,
|
||||||
|
fromSystemUid)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def gossipToProto(gossip: Gossip): dm.Gossip = {
|
private def gossipToProto(gossip: Gossip): dm.Gossip = {
|
||||||
|
|
@ -284,18 +290,24 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
case (key, data) =>
|
case (key, data) =>
|
||||||
b.addEntries(dm.Gossip.Entry.newBuilder().setKey(key).setEnvelope(dataEnvelopeToProto(data)))
|
b.addEntries(dm.Gossip.Entry.newBuilder().setKey(key).setEnvelope(dataEnvelopeToProto(data)))
|
||||||
}
|
}
|
||||||
|
b.setToSystemUid(gossip.toSystemUid.get)
|
||||||
|
b.setFromSystemUid(gossip.fromSystemUid.get)
|
||||||
b.build()
|
b.build()
|
||||||
}
|
}
|
||||||
|
|
||||||
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
||||||
val gossip = dm.Gossip.parseFrom(decompress(bytes))
|
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(
|
||||||
gossip.getEntriesList.asScala.iterator.map(e => e.getKey -> dataEnvelopeFromProto(e.getEnvelope)).toMap,
|
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 = {
|
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)
|
if (deltaPropagation.reply)
|
||||||
b.setReply(deltaPropagation.reply)
|
b.setReply(deltaPropagation.reply)
|
||||||
deltaPropagation.deltas.foreach {
|
deltaPropagation.deltas.foreach {
|
||||||
|
|
@ -513,18 +525,27 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def writeToProto(write: Write): dm.Write =
|
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 = {
|
private def writeFromBinary(bytes: Array[Byte]): Write = {
|
||||||
val write = dm.Write.parseFrom(bytes)
|
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 =
|
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 =
|
private def readFromBinary(bytes: Array[Byte]): Read = {
|
||||||
Read(dm.Read.parseFrom(bytes).getKey)
|
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 = {
|
private def readResultToProto(readResult: ReadResult): dm.ReadResult = {
|
||||||
val b = dm.ReadResult.newBuilder()
|
val b = dm.ReadResult.newBuilder()
|
||||||
|
|
|
||||||
|
|
@ -181,6 +181,12 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
||||||
values should ===(Set(10))
|
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
|
// after merging with others
|
||||||
replicator3 ! Get(KeyA, ReadAll(remainingOrDefault))
|
replicator3 ! Get(KeyA, ReadAll(remainingOrDefault))
|
||||||
val counter5 = expectMsgType[GetSuccess[GCounter]].dataValue
|
val counter5 = expectMsgType[GetSuccess[GCounter]].dataValue
|
||||||
|
|
|
||||||
|
|
@ -105,6 +105,7 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
|
||||||
expectMsg(ReplicaCount(5))
|
expectMsg(ReplicaCount(5))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
enterBarrier("all-joined")
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
for (_ <- 0 until 5) {
|
for (_ <- 0 until 5) {
|
||||||
|
|
|
||||||
|
|
@ -16,7 +16,7 @@ import org.scalatest.Matchers
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
|
|
||||||
object DeltaPropagationSelectorSpec {
|
object DeltaPropagationSelectorSpec {
|
||||||
class TestSelector(val selfUniqueAddress: UniqueAddress, override val allNodes: Vector[Address])
|
class TestSelector(val selfUniqueAddress: UniqueAddress, override val allNodes: Vector[UniqueAddress])
|
||||||
extends DeltaPropagationSelector {
|
extends DeltaPropagationSelector {
|
||||||
override val gossipIntervalDivisor = 5
|
override val gossipIntervalDivisor = 5
|
||||||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
||||||
|
|
@ -33,14 +33,14 @@ object DeltaPropagationSelectorSpec {
|
||||||
|
|
||||||
class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheckedTripleEquals {
|
class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheckedTripleEquals {
|
||||||
import DeltaPropagationSelectorSpec._
|
import DeltaPropagationSelectorSpec._
|
||||||
val selfUniqueAddress = UniqueAddress(Address("akka", "Sys", "localhost", 4999), 1L)
|
val selfUniqueAddress = UniqueAddress(Address("akka", "Sys", "localhost", 4999), 17L)
|
||||||
val nodes = (2500 until 2600).map(n => Address("akka", "Sys", "localhost", n)).toVector
|
val nodes = (2500 until 2600).map(n => UniqueAddress(Address("akka", "Sys", "localhost", n), 17L)).toVector
|
||||||
|
|
||||||
"DeltaPropagationSelector" must {
|
"DeltaPropagationSelector" must {
|
||||||
"collect none when no nodes" in {
|
"collect none when no nodes" in {
|
||||||
val selector = new TestSelector(selfUniqueAddress, Vector.empty)
|
val selector = new TestSelector(selfUniqueAddress, Vector.empty)
|
||||||
selector.update("A", deltaA)
|
selector.update("A", deltaA)
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(false)
|
selector.hasDeltaEntries("A") should ===(false)
|
||||||
}
|
}
|
||||||
|
|
@ -56,9 +56,9 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
DeltaPropagation(
|
DeltaPropagation(
|
||||||
selfUniqueAddress,
|
selfUniqueAddress,
|
||||||
false,
|
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(nodes(0) -> expected))
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(false)
|
selector.hasDeltaEntries("A") should ===(false)
|
||||||
selector.hasDeltaEntries("B") should ===(false)
|
selector.hasDeltaEntries("B") should ===(false)
|
||||||
|
|
@ -72,13 +72,13 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
DeltaPropagation(
|
DeltaPropagation(
|
||||||
selfUniqueAddress,
|
selfUniqueAddress,
|
||||||
false,
|
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, nodes(1) -> expected))
|
selector.collectPropagations() should ===(Map(nodes(0) -> expected, nodes(1) → expected))
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(true)
|
selector.hasDeltaEntries("A") should ===(true)
|
||||||
selector.hasDeltaEntries("B") should ===(true)
|
selector.hasDeltaEntries("B") should ===(true)
|
||||||
selector.collectPropagations() should ===(Map(nodes(2) -> expected))
|
selector.collectPropagations() should ===(Map(nodes(2) -> expected))
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(false)
|
selector.hasDeltaEntries("A") should ===(false)
|
||||||
selector.hasDeltaEntries("B") should ===(false)
|
selector.hasDeltaEntries("B") should ===(false)
|
||||||
|
|
@ -92,8 +92,8 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
DeltaPropagation(
|
DeltaPropagation(
|
||||||
selfUniqueAddress,
|
selfUniqueAddress,
|
||||||
false,
|
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) -> expected1, nodes(1) -> expected1))
|
selector.collectPropagations() should ===(Map(nodes(0) -> expected1, nodes(1) → expected1))
|
||||||
// new update before previous was propagated to all nodes
|
// new update before previous was propagated to all nodes
|
||||||
selector.update("C", deltaC)
|
selector.update("C", deltaC)
|
||||||
val expected2 = DeltaPropagation(
|
val expected2 = DeltaPropagation(
|
||||||
|
|
@ -103,14 +103,15 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
"A" -> Delta(DataEnvelope(deltaA), 1L, 1L),
|
"A" -> Delta(DataEnvelope(deltaA), 1L, 1L),
|
||||||
"B" -> Delta(DataEnvelope(deltaB), 1L, 1L),
|
"B" -> Delta(DataEnvelope(deltaB), 1L, 1L),
|
||||||
"C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
"C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||||
val expected3 = DeltaPropagation(selfUniqueAddress, false, Map("C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
val expected3 =
|
||||||
selector.collectPropagations() should ===(Map(nodes(2) -> expected2, nodes(0) -> expected3))
|
DeltaPropagation(selfUniqueAddress, false, Map("C" -> Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||||
|
selector.collectPropagations() should ===(Map(nodes(2) -> expected2, nodes(0) → expected3))
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(false)
|
selector.hasDeltaEntries("A") should ===(false)
|
||||||
selector.hasDeltaEntries("B") should ===(false)
|
selector.hasDeltaEntries("B") should ===(false)
|
||||||
selector.hasDeltaEntries("C") should ===(true)
|
selector.hasDeltaEntries("C") should ===(true)
|
||||||
selector.collectPropagations() should ===(Map(nodes(1) -> expected3))
|
selector.collectPropagations() should ===(Map(nodes(1) -> expected3))
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("C") should ===(false)
|
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.collectPropagations() should ===(Map(nodes(0) -> expected1))
|
||||||
selector.update("A", delta3)
|
selector.update("A", delta3)
|
||||||
selector.currentVersion("A") should ===(3L)
|
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(nodes(0) -> expected2))
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||||
}
|
}
|
||||||
|
|
||||||
"merge deltas" in {
|
"merge deltas" in {
|
||||||
|
|
@ -142,7 +144,8 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
override def nodesSliceSize(allNodesSize: Int): Int = 1
|
override def nodesSliceSize(allNodesSize: Int): Int = 1
|
||||||
}
|
}
|
||||||
selector.update("A", delta1)
|
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.collectPropagations() should ===(Map(nodes(0) -> expected1))
|
||||||
|
|
||||||
selector.update("A", delta2)
|
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)))
|
DeltaPropagation(selfUniqueAddress, false, Map("A" -> Delta(DataEnvelope(delta2.merge(delta3)), 2L, 3L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) -> expected4))
|
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(nodes(1) -> expected5))
|
||||||
|
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[UniqueAddress, DeltaPropagation])
|
||||||
}
|
}
|
||||||
|
|
||||||
"discard too large deltas" in {
|
"discard too large deltas" in {
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.ActorSelection
|
import akka.actor.ActorSelection
|
||||||
|
|
@ -15,9 +16,11 @@ import akka.testkit._
|
||||||
import akka.cluster.ddata.Replicator.Internal._
|
import akka.cluster.ddata.Replicator.Internal._
|
||||||
import akka.cluster.ddata.Replicator._
|
import akka.cluster.ddata.Replicator._
|
||||||
import akka.remote.RARP
|
import akka.remote.RARP
|
||||||
|
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
|
|
||||||
|
import akka.cluster.Cluster
|
||||||
|
import akka.cluster.UniqueAddress
|
||||||
|
|
||||||
object WriteAggregatorSpec {
|
object WriteAggregatorSpec {
|
||||||
|
|
||||||
val KeyA = GSetKey[String]("A")
|
val KeyA = GSetKey[String]("A")
|
||||||
|
|
@ -26,41 +29,76 @@ object WriteAggregatorSpec {
|
||||||
def writeAggregatorProps(
|
def writeAggregatorProps(
|
||||||
data: GSet[String],
|
data: GSet[String],
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
probes: Map[Address, ActorRef],
|
probes: Map[UniqueAddress, ActorRef],
|
||||||
nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
unreachable: Set[Address],
|
nodes: Set[UniqueAddress],
|
||||||
|
unreachable: Set[UniqueAddress],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean): Props =
|
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(
|
def writeAggregatorPropsWithDelta(
|
||||||
data: ORSet[String],
|
data: ORSet[String],
|
||||||
delta: Delta,
|
delta: Delta,
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
probes: Map[Address, ActorRef],
|
probes: Map[UniqueAddress, ActorRef],
|
||||||
nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
unreachable: Set[Address],
|
nodes: Set[UniqueAddress],
|
||||||
|
unreachable: Set[UniqueAddress],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean): Props =
|
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(
|
class TestWriteAggregator(
|
||||||
key: Key.KeyR,
|
key: Key.KeyR,
|
||||||
data: ReplicatedData,
|
data: ReplicatedData,
|
||||||
delta: Option[Delta],
|
delta: Option[Delta],
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
probes: Map[Address, ActorRef],
|
probes: Map[UniqueAddress, ActorRef],
|
||||||
nodes: Set[Address],
|
selfUniqueAddress: UniqueAddress,
|
||||||
unreachable: Set[Address],
|
nodes: Set[UniqueAddress],
|
||||||
|
unreachable: Set[UniqueAddress],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean)
|
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)
|
context.actorSelection(probes(address).path)
|
||||||
|
|
||||||
override def senderAddress(): Address =
|
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 =
|
def writeAckAdapterProps(replica: ActorRef): Props =
|
||||||
|
|
@ -105,10 +143,10 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
|
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
|
||||||
else "akka.tcp"
|
else "akka.tcp"
|
||||||
|
|
||||||
val nodeA = Address(protocol, "Sys", "a", 2552)
|
val nodeA = UniqueAddress(Address(protocol, "Sys", "a", 2552), 17L)
|
||||||
val nodeB = nodeA.copy(host = Some("b"))
|
val nodeB = UniqueAddress(Address(protocol, "Sys", "b", 2552), 17L)
|
||||||
val nodeC = nodeA.copy(host = Some("c"))
|
val nodeC = UniqueAddress(Address(protocol, "Sys", "c", 2552), 17L)
|
||||||
val nodeD = nodeA.copy(host = Some("d"))
|
val nodeD = UniqueAddress(Address(protocol, "Sys", "d", 2552), 17L)
|
||||||
// 4 replicas + the local => 5
|
// 4 replicas + the local => 5
|
||||||
val nodes = Set(nodeA, nodeB, nodeC, nodeD)
|
val nodes = Set(nodeA, nodeB, nodeC, nodeD)
|
||||||
|
|
||||||
|
|
@ -118,13 +156,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
val writeMajority = WriteMajority(timeout)
|
val writeMajority = WriteMajority(timeout)
|
||||||
val writeAll = WriteAll(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
|
nodes.toSeq.map(_ -> system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a tuple for each node with the WriteAckAdapter and the TestProbe
|
* 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
|
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 {
|
"send to at least N/2+1 replicas when WriteMajority" in {
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val aggr = system.actorOf(
|
val aggr = system.actorOf(
|
||||||
WriteAggregatorSpec
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
|
data,
|
||||||
|
writeMajority,
|
||||||
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set.empty,
|
||||||
|
testActor,
|
||||||
|
durable = false))
|
||||||
|
|
||||||
probe.expectMsgType[Write]
|
probe.expectMsgType[Write]
|
||||||
probe.lastSender ! WriteAck
|
probe.lastSender ! WriteAck
|
||||||
|
|
@ -147,8 +194,16 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
"send to more when no immediate reply" in {
|
"send to more when no immediate reply" in {
|
||||||
val testProbes = probes()
|
val testProbes = probes()
|
||||||
val testProbeRefs = testProbes.map { case (a, tm) => a -> tm.writeAckAdapter }
|
val testProbeRefs = testProbes.map { case (a, tm) => a -> tm.writeAckAdapter }
|
||||||
val aggr = system.actorOf(WriteAggregatorSpec
|
val aggr = system.actorOf(
|
||||||
.writeAggregatorProps(data, writeMajority, testProbeRefs, nodes, Set(nodeC, nodeD), testActor, durable = false))
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
|
data,
|
||||||
|
writeMajority,
|
||||||
|
testProbeRefs,
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set(nodeC, nodeD),
|
||||||
|
testActor,
|
||||||
|
durable = false))
|
||||||
|
|
||||||
testProbes(nodeA).expectMsgType[Write]
|
testProbes(nodeA).expectMsgType[Write]
|
||||||
// no reply
|
// no reply
|
||||||
|
|
@ -173,8 +228,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
"timeout when less than required acks" in {
|
"timeout when less than required acks" in {
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val aggr = system.actorOf(
|
val aggr = system.actorOf(
|
||||||
WriteAggregatorSpec
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
|
data,
|
||||||
|
writeMajority,
|
||||||
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set.empty,
|
||||||
|
testActor,
|
||||||
|
durable = false))
|
||||||
|
|
||||||
probe.expectMsgType[Write]
|
probe.expectMsgType[Write]
|
||||||
// no reply
|
// no reply
|
||||||
|
|
@ -221,6 +283,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
delta,
|
delta,
|
||||||
writeMajority,
|
writeMajority,
|
||||||
probes(probe.ref),
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
nodes,
|
nodes,
|
||||||
Set.empty,
|
Set.empty,
|
||||||
testActor,
|
testActor,
|
||||||
|
|
@ -244,6 +307,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
delta,
|
delta,
|
||||||
writeAll,
|
writeAll,
|
||||||
testProbeRefs,
|
testProbeRefs,
|
||||||
|
selfUniqueAddress,
|
||||||
nodes,
|
nodes,
|
||||||
Set.empty,
|
Set.empty,
|
||||||
testActor,
|
testActor,
|
||||||
|
|
@ -279,6 +343,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
delta,
|
delta,
|
||||||
writeAll,
|
writeAll,
|
||||||
probes(probe.ref),
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
nodes,
|
nodes,
|
||||||
Set.empty,
|
Set.empty,
|
||||||
testActor,
|
testActor,
|
||||||
|
|
@ -311,8 +376,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
"not reply before local confirmation" in {
|
"not reply before local confirmation" in {
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val aggr = system.actorOf(
|
val aggr = system.actorOf(
|
||||||
WriteAggregatorSpec
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
.writeAggregatorProps(data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
data,
|
||||||
|
writeThree,
|
||||||
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set.empty,
|
||||||
|
testActor,
|
||||||
|
durable = true))
|
||||||
|
|
||||||
probe.expectMsgType[Write]
|
probe.expectMsgType[Write]
|
||||||
probe.lastSender ! WriteAck
|
probe.lastSender ! WriteAck
|
||||||
|
|
@ -331,8 +403,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
"tolerate WriteNack if enough WriteAck" in {
|
"tolerate WriteNack if enough WriteAck" in {
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val aggr = system.actorOf(
|
val aggr = system.actorOf(
|
||||||
WriteAggregatorSpec
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
.writeAggregatorProps(data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
data,
|
||||||
|
writeThree,
|
||||||
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set.empty,
|
||||||
|
testActor,
|
||||||
|
durable = true))
|
||||||
|
|
||||||
aggr ! UpdateSuccess(WriteAggregatorSpec.KeyA, None) // the local write
|
aggr ! UpdateSuccess(WriteAggregatorSpec.KeyA, None) // the local write
|
||||||
probe.expectMsgType[Write]
|
probe.expectMsgType[Write]
|
||||||
|
|
@ -350,8 +429,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
"reply with StoreFailure when too many nacks" in {
|
"reply with StoreFailure when too many nacks" in {
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val aggr = system.actorOf(
|
val aggr = system.actorOf(
|
||||||
WriteAggregatorSpec
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
data,
|
||||||
|
writeMajority,
|
||||||
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set.empty,
|
||||||
|
testActor,
|
||||||
|
durable = true))
|
||||||
|
|
||||||
probe.expectMsgType[Write]
|
probe.expectMsgType[Write]
|
||||||
probe.lastSender ! WriteNack
|
probe.lastSender ! WriteNack
|
||||||
|
|
@ -371,8 +457,15 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||||
"timeout when less than required acks" in {
|
"timeout when less than required acks" in {
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val aggr = system.actorOf(
|
val aggr = system.actorOf(
|
||||||
WriteAggregatorSpec
|
WriteAggregatorSpec.writeAggregatorProps(
|
||||||
.writeAggregatorProps(data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
data,
|
||||||
|
writeMajority,
|
||||||
|
probes(probe.ref),
|
||||||
|
selfUniqueAddress,
|
||||||
|
nodes,
|
||||||
|
Set.empty,
|
||||||
|
testActor,
|
||||||
|
durable = true))
|
||||||
|
|
||||||
probe.expectMsgType[Write]
|
probe.expectMsgType[Write]
|
||||||
// no reply
|
// no reply
|
||||||
|
|
|
||||||
|
|
@ -103,17 +103,26 @@ class ReplicatorMessageSerializerSpec
|
||||||
pruning = Map(
|
pruning = Map(
|
||||||
address1 -> PruningPerformed(System.currentTimeMillis()),
|
address1 -> PruningPerformed(System.currentTimeMillis()),
|
||||||
address3 -> PruningInitialized(address2, Set(address1.address)))))
|
address3 -> PruningInitialized(address2, Set(address1.address)))))
|
||||||
checkSerialization(Write("A", DataEnvelope(data1)))
|
checkSerialization(Write("A", DataEnvelope(data1), Some(address1)))
|
||||||
checkSerialization(WriteAck)
|
checkSerialization(WriteAck)
|
||||||
checkSerialization(WriteNack)
|
checkSerialization(WriteNack)
|
||||||
checkSerialization(DeltaNack)
|
checkSerialization(DeltaNack)
|
||||||
checkSerialization(Read("A"))
|
checkSerialization(Read("A", Some(address1)))
|
||||||
checkSerialization(ReadResult(Some(DataEnvelope(data1))))
|
checkSerialization(ReadResult(Some(DataEnvelope(data1))))
|
||||||
checkSerialization(ReadResult(None))
|
checkSerialization(ReadResult(None))
|
||||||
checkSerialization(
|
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(
|
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(
|
checkSerialization(
|
||||||
DeltaPropagation(
|
DeltaPropagation(
|
||||||
address1,
|
address1,
|
||||||
|
|
@ -153,10 +162,10 @@ class ReplicatorMessageSerializerSpec
|
||||||
|
|
||||||
"get added element" in {
|
"get added element" in {
|
||||||
val cache = new SmallCache[Read, String](2, 5.seconds, _ => null)
|
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.add(a, "A")
|
||||||
cache.get(a) should be("A")
|
cache.get(a) should be("A")
|
||||||
val b = Read("b")
|
val b = Read("b", Some(address1))
|
||||||
cache.add(b, "B")
|
cache.add(b, "B")
|
||||||
cache.get(a) should be("A")
|
cache.get(a) should be("A")
|
||||||
cache.get(b) should be("B")
|
cache.get(b) should be("B")
|
||||||
|
|
@ -164,20 +173,20 @@ class ReplicatorMessageSerializerSpec
|
||||||
|
|
||||||
"return null for non-existing elements" in {
|
"return null for non-existing elements" in {
|
||||||
val cache = new SmallCache[Read, String](4, 5.seconds, _ => null)
|
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.get(a) should be(null)
|
||||||
cache.add(a, "A")
|
cache.add(a, "A")
|
||||||
val b = Read("b")
|
val b = Read("b", Some(address1))
|
||||||
cache.get(b) should be(null)
|
cache.get(b) should be(null)
|
||||||
}
|
}
|
||||||
|
|
||||||
"hold latest added elements" in {
|
"hold latest added elements" in {
|
||||||
val cache = new SmallCache[Read, String](4, 5.seconds, _ => null)
|
val cache = new SmallCache[Read, String](4, 5.seconds, _ => null)
|
||||||
val a = Read("a")
|
val a = Read("a", Some(address1))
|
||||||
val b = Read("b")
|
val b = Read("b", Some(address1))
|
||||||
val c = Read("c")
|
val c = Read("c", Some(address1))
|
||||||
val d = Read("d")
|
val d = Read("d", Some(address1))
|
||||||
val e = Read("e")
|
val e = Read("e", Some(address1))
|
||||||
cache.add(a, "A")
|
cache.add(a, "A")
|
||||||
cache.get(a) should be("A")
|
cache.get(a) should be("A")
|
||||||
cache.add(b, "B")
|
cache.add(b, "B")
|
||||||
|
|
@ -204,7 +213,7 @@ class ReplicatorMessageSerializerSpec
|
||||||
|
|
||||||
"handle Int wrap around" ignore { // ignored because it takes 20 seconds (but it works)
|
"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 cache = new SmallCache[Read, String](2, 5.seconds, _ => null)
|
||||||
val a = Read("a")
|
val a = Read("a", Some(address1))
|
||||||
val x = a -> "A"
|
val x = a -> "A"
|
||||||
var n = 0
|
var n = 0
|
||||||
while (n <= Int.MaxValue - 3) {
|
while (n <= Int.MaxValue - 3) {
|
||||||
|
|
@ -214,8 +223,8 @@ class ReplicatorMessageSerializerSpec
|
||||||
|
|
||||||
cache.get(a) should be("A")
|
cache.get(a) should be("A")
|
||||||
|
|
||||||
val b = Read("b")
|
val b = Read("b", Some(address1))
|
||||||
val c = Read("c")
|
val c = Read("c", Some(address1))
|
||||||
cache.add(b, "B")
|
cache.add(b, "B")
|
||||||
cache.get(a) should be("A")
|
cache.get(a) should be("A")
|
||||||
cache.get(b) should be("B")
|
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 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)
|
val v1 = cache.getOrAdd(a)
|
||||||
v1.toString should be("v1")
|
v1.toString should be("v1")
|
||||||
(cache.getOrAdd(a) should be).theSameInstanceAs(v1)
|
(cache.getOrAdd(a) should be).theSameInstanceAs(v1)
|
||||||
|
|
@ -249,8 +258,8 @@ class ReplicatorMessageSerializerSpec
|
||||||
|
|
||||||
"evict cache after time-to-live" in {
|
"evict cache after time-to-live" in {
|
||||||
val cache = new SmallCache[Read, AnyRef](4, 10.millis, _ => null)
|
val cache = new SmallCache[Read, AnyRef](4, 10.millis, _ => null)
|
||||||
val b = Read("b")
|
val b = Read("b", Some(address1))
|
||||||
val c = Read("c")
|
val c = Read("c", Some(address1))
|
||||||
cache.add(b, "B")
|
cache.add(b, "B")
|
||||||
cache.add(c, "C")
|
cache.add(c, "C")
|
||||||
|
|
||||||
|
|
@ -262,8 +271,8 @@ class ReplicatorMessageSerializerSpec
|
||||||
|
|
||||||
"not evict cache before time-to-live" in {
|
"not evict cache before time-to-live" in {
|
||||||
val cache = new SmallCache[Read, AnyRef](4, 5.seconds, _ => null)
|
val cache = new SmallCache[Read, AnyRef](4, 5.seconds, _ => null)
|
||||||
val b = Read("b")
|
val b = Read("b", Some(address1))
|
||||||
val c = Read("c")
|
val c = Read("c", Some(address1))
|
||||||
cache.add(b, "B")
|
cache.add(b, "B")
|
||||||
cache.add(c, "C")
|
cache.add(c, "C")
|
||||||
cache.evict()
|
cache.evict()
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue