Make cluster sharding DC aware, #23231

* Sharding only within own team (coordinator is singleton)
* the ddata Replicator used by Sharding must also be only within own team
* added support for Set of roles in ddata Replicator so that can be used
  by sharding to specify role + team
* Sharding proxy can route to sharding in another team
This commit is contained in:
Patrik Nordwall 2017-06-26 15:03:33 +02:00
parent e37243f471
commit e0fe0bc49e
12 changed files with 417 additions and 47 deletions

View file

@ -34,7 +34,7 @@ class DistributedData(system: ExtendedActorSystem) extends Extension {
* Returns true if this member is not tagged with the role configured for the
* replicas.
*/
def isTerminated: Boolean = Cluster(system).isTerminated || !settings.role.forall(Cluster(system).selfRoles.contains)
def isTerminated: Boolean = Cluster(system).isTerminated || !settings.roles.subsetOf(Cluster(system).selfRoles)
/**
* `ActorRef` of the [[Replicator]] .

View file

@ -48,6 +48,9 @@ import akka.actor.Cancellable
import scala.util.control.NonFatal
import akka.cluster.ddata.Key.KeyId
import akka.annotation.InternalApi
import scala.collection.immutable.TreeSet
import akka.cluster.MemberStatus
import scala.annotation.varargs
object ReplicatorSettings {
@ -98,8 +101,8 @@ object ReplicatorSettings {
}
/**
* @param role Replicas are running on members tagged with this role.
* All members are used if undefined.
* @param roles Replicas are running on members tagged with these roles.
* The member must have all given roles. All members are used if empty.
* @param gossipInterval How often the Replicator should send out gossip information.
* @param notifySubscribersInterval How often the subscribers will be notified
* of changes, if any.
@ -124,7 +127,7 @@ object ReplicatorSettings {
* in the `Set`.
*/
final class ReplicatorSettings(
val role: Option[String],
val roles: Set[String],
val gossipInterval: FiniteDuration,
val notifySubscribersInterval: FiniteDuration,
val maxDeltaElements: Int,
@ -138,10 +141,29 @@ final class ReplicatorSettings(
val deltaCrdtEnabled: Boolean,
val maxDeltaSize: Int) {
// for backwards compatibility
def this(
role: Option[String],
gossipInterval: FiniteDuration,
notifySubscribersInterval: FiniteDuration,
maxDeltaElements: Int,
dispatcher: String,
pruningInterval: FiniteDuration,
maxPruningDissemination: FiniteDuration,
durableStoreProps: Either[(String, Config), Props],
durableKeys: Set[KeyId],
pruningMarkerTimeToLive: FiniteDuration,
durablePruningMarkerTimeToLive: FiniteDuration,
deltaCrdtEnabled: Boolean,
maxDeltaSize: Int) =
this(role.toSet, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
maxPruningDissemination, durableStoreProps, durableKeys, pruningMarkerTimeToLive, durablePruningMarkerTimeToLive,
deltaCrdtEnabled, maxDeltaSize)
// For backwards compatibility
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration) =
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
this(roles = role.toSet, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days, true, 200)
// For backwards compatibility
@ -161,9 +183,20 @@ final class ReplicatorSettings(
maxPruningDissemination, durableStoreProps, durableKeys, pruningMarkerTimeToLive, durablePruningMarkerTimeToLive,
deltaCrdtEnabled, 200)
def withRole(role: String): ReplicatorSettings = copy(role = ReplicatorSettings.roleOption(role))
def withRole(role: String): ReplicatorSettings = copy(roles = ReplicatorSettings.roleOption(role).toSet)
def withRole(role: Option[String]): ReplicatorSettings = copy(role = role)
def withRole(role: Option[String]): ReplicatorSettings = copy(roles = role.toSet)
@varargs
def withRoles(roles: String*): ReplicatorSettings = copy(roles = roles.toSet)
/**
* INTERNAL API
*/
@InternalApi private[akka] def withRoles(roles: Set[String]): ReplicatorSettings = copy(roles = roles)
// for backwards compatibility
def role: Option[String] = roles.headOption
def withGossipInterval(gossipInterval: FiniteDuration): ReplicatorSettings =
copy(gossipInterval = gossipInterval)
@ -216,7 +249,7 @@ final class ReplicatorSettings(
copy(maxDeltaSize = maxDeltaSize)
private def copy(
role: Option[String] = role,
roles: Set[String] = roles,
gossipInterval: FiniteDuration = gossipInterval,
notifySubscribersInterval: FiniteDuration = notifySubscribersInterval,
maxDeltaElements: Int = maxDeltaElements,
@ -229,7 +262,7 @@ final class ReplicatorSettings(
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
deltaCrdtEnabled: Boolean = deltaCrdtEnabled,
maxDeltaSize: Int = maxDeltaSize): ReplicatorSettings =
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
new ReplicatorSettings(roles, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys,
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive, deltaCrdtEnabled, maxDeltaSize)
}
@ -988,8 +1021,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
require(!cluster.isTerminated, "Cluster node must not be terminated")
require(
role.forall(cluster.selfRoles.contains),
s"This cluster member [${selfAddress}] doesn't have the role [$role]")
roles.subsetOf(cluster.selfRoles),
s"This cluster member [${selfAddress}] doesn't have all the roles [${roles.mkString(", ")}]")
//Start periodic gossip to random nodes in cluster
import context.dispatcher
@ -1057,8 +1090,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
var weaklyUpNodes: Set[Address] = Set.empty
var removedNodes: Map[UniqueAddress, Long] = Map.empty
var leader: Option[Address] = None
def isLeader: Boolean = leader.exists(_ == selfAddress)
// all nodes sorted with the leader first
var leader: TreeSet[Member] = TreeSet.empty(Member.leaderStatusOrdering)
def isLeader: Boolean =
leader.nonEmpty && leader.head.address == selfAddress && leader.head.status == MemberStatus.Up
// for pruning timeouts are based on clock that is only increased when all nodes are reachable
var previousClockTime = System.nanoTime()
@ -1099,9 +1134,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
override def preStart(): Unit = {
if (hasDurableKeys)
durableStore ! LoadAll
val leaderChangedClass = if (role.isDefined) classOf[RoleLeaderChanged] else classOf[LeaderChanged]
// not using LeaderChanged/RoleLeaderChanged because here we need one node independent of team
cluster.subscribe(self, initialStateMode = InitialStateAsEvents,
classOf[MemberEvent], classOf[ReachabilityEvent], leaderChangedClass)
classOf[MemberEvent], classOf[ReachabilityEvent])
}
override def postStop(): Unit = {
@ -1113,7 +1148,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
clockTask.cancel()
}
def matchingRole(m: Member): Boolean = role.forall(m.hasRole)
def matchingRole(m: Member): Boolean = roles.subsetOf(m.roles)
override val supervisorStrategy = {
def fromDurableStore: Boolean = sender() == durableStore && sender() != context.system.deadLetters
@ -1204,11 +1239,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
case MemberWeaklyUp(m) receiveWeaklyUpMemberUp(m)
case MemberUp(m) receiveMemberUp(m)
case MemberRemoved(m, _) receiveMemberRemoved(m)
case _: MemberEvent // not of interest
case evt: MemberEvent receiveOtherMemberEvent(evt.member)
case UnreachableMember(m) receiveUnreachable(m)
case ReachableMember(m) receiveReachable(m)
case LeaderChanged(leader) receiveLeaderChanged(leader, None)
case RoleLeaderChanged(role, leader) receiveLeaderChanged(leader, Some(role))
case GetKeyIds receiveGetKeyIds()
case Delete(key, consistency, req) receiveDelete(key, consistency, req)
case RemovedNodePruningTick receiveRemovedNodePruningTick()
@ -1695,15 +1728,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
weaklyUpNodes += m.address
def receiveMemberUp(m: Member): Unit =
if (matchingRole(m) && m.address != selfAddress) {
nodes += m.address
weaklyUpNodes -= m.address
if (matchingRole(m)) {
leader += m
if (m.address != selfAddress) {
nodes += m.address
weaklyUpNodes -= m.address
}
}
def receiveMemberRemoved(m: Member): Unit = {
if (m.address == selfAddress)
context stop self
else if (matchingRole(m)) {
leader -= m
nodes -= m.address
weaklyUpNodes -= m.address
log.debug("adding removed node [{}] from MemberRemoved", m.uniqueAddress)
@ -1713,15 +1750,18 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
}
def receiveOtherMemberEvent(m: Member): Unit =
if (matchingRole(m)) {
// update changed status
leader = (leader - m) + m
}
def receiveUnreachable(m: Member): Unit =
if (matchingRole(m)) unreachable += m.address
def receiveReachable(m: Member): Unit =
if (matchingRole(m)) unreachable -= m.address
def receiveLeaderChanged(leaderOption: Option[Address], roleOption: Option[String]): Unit =
if (roleOption == role) leader = leaderOption
def receiveClockTick(): Unit = {
val now = System.nanoTime()
if (unreachable.isEmpty)

View file

@ -16,6 +16,7 @@ import com.typesafe.config.ConfigFactory
import akka.actor.ActorSystem
import akka.actor.ActorRef
import scala.concurrent.Await
import akka.cluster.MemberStatus
object DurablePruningSpec extends MultiNodeConfig {
val first = role("first")
@ -73,6 +74,13 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
val replicator2 = startReplicator(sys2)
val probe2 = TestProbe()(sys2)
Cluster(sys2).join(node(first).address)
awaitAssert({
Cluster(system).state.members.size should ===(4)
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up))
Cluster(sys2).state.members.size should ===(4)
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up))
}, 10.seconds)
enterBarrier("joined")
within(5.seconds) {
awaitAssert {