2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2013-01-09 01:47:48 +01:00
|
|
|
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
|
|
|
|
package akka.cluster
|
|
|
|
|
|
2012-07-06 17:04:04 +02:00
|
|
|
import language.postfixOps
|
|
|
|
|
|
2012-11-15 12:33:11 +01:00
|
|
|
import scala.collection.immutable
|
2012-09-21 14:50:06 +02:00
|
|
|
import scala.concurrent.duration._
|
2013-03-26 18:17:50 +01:00
|
|
|
import akka.actor.{ ActorLogging, ActorRef, ActorSelection, Address, Actor, RootActorPath, Props }
|
2012-10-01 14:12:20 +02:00
|
|
|
import akka.cluster.ClusterEvent._
|
2013-01-15 09:35:07 +01:00
|
|
|
import akka.routing.MurmurHash
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
2012-10-01 14:12:20 +02:00
|
|
|
* INTERNAL API
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2012-10-01 14:12:20 +02:00
|
|
|
private[akka] object ClusterHeartbeatReceiver {
|
|
|
|
|
/**
|
|
|
|
|
* Sent at regular intervals for failure detection.
|
|
|
|
|
*/
|
|
|
|
|
case class Heartbeat(from: Address) extends ClusterMessage
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Tell failure detector at receiving side that it should
|
|
|
|
|
* remove the monitoring, because heartbeats will end from
|
|
|
|
|
* this node.
|
|
|
|
|
*/
|
|
|
|
|
case class EndHeartbeat(from: Address) extends ClusterMessage
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
*
|
2012-10-01 10:02:48 +02:00
|
|
|
* Receives Heartbeat messages and updates failure detector.
|
2012-07-05 13:55:08 +02:00
|
|
|
* Instantiated as a single instance for each Cluster - e.g. heartbeats are serialized
|
|
|
|
|
* to Cluster message after message, but concurrent with other types of messages.
|
|
|
|
|
*/
|
2012-10-01 10:02:48 +02:00
|
|
|
private[cluster] final class ClusterHeartbeatReceiver extends Actor with ActorLogging {
|
2012-10-01 14:12:20 +02:00
|
|
|
import ClusterHeartbeatReceiver._
|
2012-09-06 21:48:40 +02:00
|
|
|
|
|
|
|
|
val failureDetector = Cluster(context.system).failureDetector
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
def receive = {
|
2012-10-01 14:12:20 +02:00
|
|
|
case Heartbeat(from) ⇒ failureDetector heartbeat from
|
|
|
|
|
case EndHeartbeat(from) ⇒ failureDetector remove from
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] object ClusterHeartbeatSender {
|
|
|
|
|
/**
|
2013-01-15 09:35:07 +01:00
|
|
|
* Request heartbeats from another node. Sent from the node that is
|
|
|
|
|
* expecting heartbeats from a specific sender, but has not received any.
|
|
|
|
|
*/
|
|
|
|
|
case class HeartbeatRequest(from: Address) extends ClusterMessage
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Delayed sending of a HeartbeatRequest. The actual request is
|
|
|
|
|
* only sent if no expected heartbeat message has been received.
|
|
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
|
|
|
|
case class SendHeartbeatRequest(to: Address)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Trigger a fake heartbeat message to trigger start of failure detection
|
|
|
|
|
* of a node that this node is expecting heartbeats from. HeartbeatRequest
|
|
|
|
|
* has been sent to the node so it should have started sending heartbeat
|
|
|
|
|
* messages.
|
2012-07-05 13:55:08 +02:00
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
2013-01-15 09:35:07 +01:00
|
|
|
case class ExpectedFirstHeartbeat(from: Address)
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/*
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* This actor is responsible for sending the heartbeat messages to
|
2012-10-01 14:12:20 +02:00
|
|
|
* a few other nodes that will monitor this node.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2012-09-06 21:48:40 +02:00
|
|
|
private[cluster] final class ClusterHeartbeatSender extends Actor with ActorLogging {
|
2012-07-05 13:55:08 +02:00
|
|
|
import ClusterHeartbeatSender._
|
2012-10-01 14:12:20 +02:00
|
|
|
import ClusterHeartbeatReceiver._
|
2012-10-01 10:02:48 +02:00
|
|
|
import InternalClusterAction.HeartbeatTick
|
|
|
|
|
|
|
|
|
|
val cluster = Cluster(context.system)
|
|
|
|
|
import cluster.{ selfAddress, scheduler }
|
|
|
|
|
import cluster.settings._
|
2013-05-23 13:36:35 +02:00
|
|
|
import cluster.InfoLogger._
|
2012-10-01 10:02:48 +02:00
|
|
|
import context.dispatcher
|
|
|
|
|
|
|
|
|
|
val selfHeartbeat = Heartbeat(selfAddress)
|
2012-10-01 14:12:20 +02:00
|
|
|
val selfEndHeartbeat = EndHeartbeat(selfAddress)
|
2013-01-15 09:35:07 +01:00
|
|
|
val selfHeartbeatRequest = HeartbeatRequest(selfAddress)
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
var state = ClusterHeartbeatSenderState.empty(selfAddress, MonitoredByNrOfMembers)
|
2012-10-01 10:02:48 +02:00
|
|
|
|
|
|
|
|
// start periodic heartbeat to other nodes in cluster
|
2012-10-15 17:17:54 +02:00
|
|
|
val heartbeatTask = scheduler.schedule(PeriodicTasksInitialDelay max HeartbeatInterval,
|
2012-10-09 18:11:36 +02:00
|
|
|
HeartbeatInterval, self, HeartbeatTick)
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2012-11-27 18:07:37 +01:00
|
|
|
override def preStart(): Unit = {
|
2013-03-05 21:05:11 +01:00
|
|
|
cluster.subscribe(self, classOf[MemberEvent])
|
2012-11-27 18:07:37 +01:00
|
|
|
cluster.subscribe(self, classOf[UnreachableMember])
|
|
|
|
|
}
|
2012-10-01 10:02:48 +02:00
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
heartbeatTask.cancel()
|
|
|
|
|
cluster.unsubscribe(self)
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Looks up and returns the remote cluster heartbeat connection for the specific address.
|
|
|
|
|
*/
|
2013-03-26 18:17:50 +01:00
|
|
|
def heartbeatReceiver(address: Address): ActorSelection =
|
|
|
|
|
context.actorSelection(RootActorPath(address) / "system" / "cluster" / "heartbeatReceiver")
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
/**
|
|
|
|
|
* Looks up and returns the remote cluster heartbeat sender for the specific address.
|
|
|
|
|
*/
|
2013-03-26 18:17:50 +01:00
|
|
|
def heartbeatSender(address: Address): ActorSelection =
|
|
|
|
|
context.actorSelection(self.path.toStringWithAddress(address))
|
2013-01-15 09:35:07 +01:00
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
def receive = {
|
2013-01-15 09:35:07 +01:00
|
|
|
case HeartbeatTick ⇒ heartbeat()
|
2013-03-05 21:05:11 +01:00
|
|
|
case MemberUp(m) ⇒ addMember(m)
|
2013-01-15 09:35:07 +01:00
|
|
|
case UnreachableMember(m) ⇒ removeMember(m)
|
2013-05-23 11:09:32 +02:00
|
|
|
case MemberRemoved(m, _) ⇒ removeMember(m)
|
2013-03-05 21:05:11 +01:00
|
|
|
case s: CurrentClusterState ⇒ reset(s)
|
2013-05-09 09:49:59 +02:00
|
|
|
case MemberExited(m) ⇒ memberExited(m)
|
2013-03-05 21:05:11 +01:00
|
|
|
case _: MemberEvent ⇒ // not interested in other types of MemberEvent
|
2013-01-15 09:35:07 +01:00
|
|
|
case HeartbeatRequest(from) ⇒ addHeartbeatRequest(from)
|
|
|
|
|
case SendHeartbeatRequest(to) ⇒ sendHeartbeatRequest(to)
|
|
|
|
|
case ExpectedFirstHeartbeat(from) ⇒ triggerFirstHeartbeat(from)
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2013-05-09 09:49:59 +02:00
|
|
|
def reset(snapshot: CurrentClusterState): Unit =
|
|
|
|
|
state = state.reset(snapshot.members.map(_.address)(collection.breakOut))
|
2012-10-01 14:12:20 +02:00
|
|
|
|
2013-01-17 14:00:01 +01:00
|
|
|
def addMember(m: Member): Unit = if (m.address != selfAddress) state = state addMember m.address
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2013-05-09 09:49:59 +02:00
|
|
|
def removeMember(m: Member): Unit = {
|
|
|
|
|
if (m.uniqueAddress == cluster.selfUniqueAddress)
|
|
|
|
|
// This cluster node will be shutdown, but stop this actor immediately
|
|
|
|
|
// to prevent it from sending out anything more and avoid sending EndHeartbeat.
|
|
|
|
|
context stop self
|
|
|
|
|
else
|
|
|
|
|
state = state removeMember m.address
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def memberExited(m: Member): Unit =
|
|
|
|
|
if (m.uniqueAddress == cluster.selfUniqueAddress) {
|
|
|
|
|
// This cluster node will be shutdown, but stop this actor immediately
|
|
|
|
|
// to prevent it from sending out anything more and avoid sending EndHeartbeat.
|
|
|
|
|
context stop self
|
|
|
|
|
}
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2013-01-17 14:00:01 +01:00
|
|
|
def addHeartbeatRequest(address: Address): Unit =
|
|
|
|
|
if (address != selfAddress) state = state.addHeartbeatRequest(address, Deadline.now + HeartbeatRequestTimeToLive)
|
2013-01-15 09:35:07 +01:00
|
|
|
|
|
|
|
|
def sendHeartbeatRequest(address: Address): Unit =
|
|
|
|
|
if (!cluster.failureDetector.isMonitoring(address) && state.ring.mySenders.contains(address)) {
|
2013-01-25 15:03:52 +01:00
|
|
|
heartbeatSender(address) ! selfHeartbeatRequest
|
2013-01-15 09:35:07 +01:00
|
|
|
// schedule the expected heartbeat for later, which will give the
|
|
|
|
|
// sender a chance to start heartbeating, and also trigger some resends of
|
|
|
|
|
// the heartbeat request
|
|
|
|
|
scheduler.scheduleOnce(HeartbeatExpectedResponseAfter, self, ExpectedFirstHeartbeat(address))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def triggerFirstHeartbeat(address: Address): Unit =
|
|
|
|
|
if (!cluster.failureDetector.isMonitoring(address)) {
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Trigger extra expected heartbeat from [{}]", address)
|
2013-01-15 09:35:07 +01:00
|
|
|
cluster.failureDetector.heartbeat(address)
|
|
|
|
|
}
|
2012-10-01 10:02:48 +02:00
|
|
|
|
|
|
|
|
def heartbeat(): Unit = {
|
2013-01-15 09:35:07 +01:00
|
|
|
state = state.removeOverdueHeartbeatRequest()
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2013-01-25 15:03:52 +01:00
|
|
|
state.active foreach { to ⇒
|
|
|
|
|
log.debug("Cluster Node [{}] - Heartbeat to [{}]", cluster.selfAddress, to)
|
|
|
|
|
heartbeatReceiver(to) ! selfHeartbeat
|
2012-10-01 10:02:48 +02:00
|
|
|
}
|
2012-10-01 14:12:20 +02:00
|
|
|
|
|
|
|
|
// When sending heartbeats to a node is stopped a few `EndHeartbeat` messages is
|
|
|
|
|
// sent to notify it that no more heartbeats will be sent.
|
2012-10-10 15:23:18 +02:00
|
|
|
for ((to, count) ← state.ending) {
|
2013-01-25 15:03:52 +01:00
|
|
|
log.debug("Cluster Node [{}] - EndHeartbeat to [{}]", cluster.selfAddress, to)
|
|
|
|
|
heartbeatReceiver(to) ! selfEndHeartbeat
|
|
|
|
|
if (count == NumberOfEndHeartbeats)
|
2012-10-10 15:23:18 +02:00
|
|
|
state = state.removeEnding(to)
|
2013-01-25 15:03:52 +01:00
|
|
|
else
|
2012-10-10 15:23:18 +02:00
|
|
|
state = state.increaseEndingCount(to)
|
2012-10-01 14:12:20 +02:00
|
|
|
}
|
2013-01-15 09:35:07 +01:00
|
|
|
|
|
|
|
|
// request heartbeats from expected sender node if no heartbeat messages has been received
|
|
|
|
|
state.ring.mySenders foreach { address ⇒
|
|
|
|
|
if (!cluster.failureDetector.isMonitoring(address))
|
|
|
|
|
scheduler.scheduleOnce(HeartbeatRequestDelay, self, SendHeartbeatRequest(address))
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2012-10-10 15:23:18 +02:00
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] object ClusterHeartbeatSenderState {
|
2012-10-01 14:12:20 +02:00
|
|
|
/**
|
2012-10-10 15:23:18 +02:00
|
|
|
* Initial, empty state
|
|
|
|
|
*/
|
2013-01-15 09:35:07 +01:00
|
|
|
def empty(selfAddress: Address, monitoredByNrOfMembers: Int): ClusterHeartbeatSenderState =
|
|
|
|
|
ClusterHeartbeatSenderState(HeartbeatNodeRing(selfAddress, Set(selfAddress), monitoredByNrOfMembers))
|
2012-10-10 15:23:18 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Create a new state based on previous state, and
|
2012-10-01 14:12:20 +02:00
|
|
|
* keep track of which nodes to stop sending heartbeats to.
|
|
|
|
|
*/
|
2012-10-10 15:23:18 +02:00
|
|
|
private def apply(
|
|
|
|
|
old: ClusterHeartbeatSenderState,
|
2013-01-15 09:35:07 +01:00
|
|
|
ring: HeartbeatNodeRing): ClusterHeartbeatSenderState = {
|
2012-10-10 15:23:18 +02:00
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
val curr = ring.myReceivers
|
2012-10-01 14:12:20 +02:00
|
|
|
// start ending process for nodes not selected any more
|
|
|
|
|
// abort ending process for nodes that have been selected again
|
2012-10-10 15:23:18 +02:00
|
|
|
val end = old.ending ++ (old.current -- curr).map(_ -> 0) -- curr
|
2013-01-15 09:35:07 +01:00
|
|
|
old.copy(ring = ring, current = curr, ending = end, heartbeatRequest = old.heartbeatRequest -- curr)
|
2012-10-01 10:02:48 +02:00
|
|
|
}
|
|
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
2012-07-05 13:55:08 +02:00
|
|
|
*
|
2012-10-10 15:23:18 +02:00
|
|
|
* State used by [akka.cluster.ClusterHeartbeatSender].
|
|
|
|
|
* The initial state is created with `empty` in the of
|
|
|
|
|
* the companion object, thereafter the state is modified
|
|
|
|
|
* with the methods, such as `addMember`. It is immutable,
|
|
|
|
|
* i.e. the methods return new instances.
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] case class ClusterHeartbeatSenderState private (
|
2013-01-15 09:35:07 +01:00
|
|
|
ring: HeartbeatNodeRing,
|
2012-10-10 15:23:18 +02:00
|
|
|
current: Set[Address] = Set.empty,
|
|
|
|
|
ending: Map[Address, Int] = Map.empty,
|
2013-01-15 09:35:07 +01:00
|
|
|
heartbeatRequest: Map[Address, Deadline] = Map.empty) {
|
2012-10-10 15:23:18 +02:00
|
|
|
|
2013-05-28 09:02:03 +02:00
|
|
|
// TODO can be disabled as optimization
|
2013-03-28 23:45:48 +01:00
|
|
|
assertInvariants()
|
2012-10-10 15:23:18 +02:00
|
|
|
|
2013-03-28 23:45:48 +01:00
|
|
|
private def assertInvariants(): Unit = {
|
2012-10-10 15:23:18 +02:00
|
|
|
val currentAndEnding = current.intersect(ending.keySet)
|
|
|
|
|
require(currentAndEnding.isEmpty,
|
2013-01-15 09:35:07 +01:00
|
|
|
s"Same nodes in current and ending not allowed, got [${currentAndEnding}]")
|
|
|
|
|
|
|
|
|
|
val currentAndHeartbeatRequest = current.intersect(heartbeatRequest.keySet)
|
|
|
|
|
require(currentAndHeartbeatRequest.isEmpty,
|
|
|
|
|
s"Same nodes in current and heartbeatRequest not allowed, got [${currentAndHeartbeatRequest}]")
|
|
|
|
|
|
|
|
|
|
val currentNotInAll = current -- ring.nodes
|
|
|
|
|
require(current.isEmpty || currentNotInAll.isEmpty,
|
|
|
|
|
s"Nodes in current but not in ring nodes not allowed, got [${currentNotInAll}]")
|
|
|
|
|
|
|
|
|
|
require(!current.contains(ring.selfAddress),
|
|
|
|
|
s"Self in current not allowed, got [${ring.selfAddress}]")
|
|
|
|
|
require(!heartbeatRequest.contains(ring.selfAddress),
|
|
|
|
|
s"Self in heartbeatRequest not allowed, got [${ring.selfAddress}]")
|
2012-10-10 15:23:18 +02:00
|
|
|
}
|
|
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
val active: Set[Address] = current ++ heartbeatRequest.keySet
|
2012-10-10 15:23:18 +02:00
|
|
|
|
2013-05-09 09:49:59 +02:00
|
|
|
def reset(nodes: immutable.HashSet[Address]): ClusterHeartbeatSenderState = {
|
2013-01-15 09:35:07 +01:00
|
|
|
ClusterHeartbeatSenderState(nodes.foldLeft(this) { _ removeHeartbeatRequest _ }, ring.copy(nodes = nodes + ring.selfAddress))
|
|
|
|
|
}
|
2012-10-10 15:23:18 +02:00
|
|
|
|
|
|
|
|
def addMember(a: Address): ClusterHeartbeatSenderState =
|
2013-01-15 09:35:07 +01:00
|
|
|
ClusterHeartbeatSenderState(removeHeartbeatRequest(a), ring :+ a)
|
2012-10-10 15:23:18 +02:00
|
|
|
|
|
|
|
|
def removeMember(a: Address): ClusterHeartbeatSenderState =
|
2013-01-15 09:35:07 +01:00
|
|
|
ClusterHeartbeatSenderState(removeHeartbeatRequest(a), ring :- a)
|
2012-10-10 15:23:18 +02:00
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
private def removeHeartbeatRequest(address: Address): ClusterHeartbeatSenderState = {
|
|
|
|
|
if (heartbeatRequest contains address)
|
|
|
|
|
copy(heartbeatRequest = heartbeatRequest - address, ending = ending + (address -> 0))
|
2012-10-10 15:23:18 +02:00
|
|
|
else this
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
def addHeartbeatRequest(address: Address, deadline: Deadline): ClusterHeartbeatSenderState = {
|
|
|
|
|
if (current.contains(address)) this
|
|
|
|
|
else copy(heartbeatRequest = heartbeatRequest + (address -> deadline), ending = ending - address)
|
2012-10-01 14:12:20 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2013-01-15 09:35:07 +01:00
|
|
|
* Cleanup overdue heartbeatRequest
|
2012-10-01 10:02:48 +02:00
|
|
|
*/
|
2013-01-15 09:35:07 +01:00
|
|
|
def removeOverdueHeartbeatRequest(): ClusterHeartbeatSenderState = {
|
|
|
|
|
val overdue = heartbeatRequest collect { case (address, deadline) if deadline.isOverdue ⇒ address }
|
2012-10-10 15:23:18 +02:00
|
|
|
if (overdue.isEmpty) this
|
|
|
|
|
else
|
2013-01-15 09:35:07 +01:00
|
|
|
copy(ending = ending ++ overdue.map(_ -> 0), heartbeatRequest = heartbeatRequest -- overdue)
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
def removeEnding(a: Address): ClusterHeartbeatSenderState = copy(ending = ending - a)
|
|
|
|
|
|
|
|
|
|
def increaseEndingCount(a: Address): ClusterHeartbeatSenderState = copy(ending = ending + (a -> (ending(a) + 1)))
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2013-01-15 09:35:07 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Data structure for picking heartbeat receivers and keep track of what nodes
|
|
|
|
|
* that are expected to send heartbeat messages to a node. The node ring is
|
|
|
|
|
* shuffled by deterministic hashing to avoid picking physically co-located
|
|
|
|
|
* neighbors.
|
|
|
|
|
*
|
|
|
|
|
* It is immutable, i.e. the methods return new instances.
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] case class HeartbeatNodeRing(selfAddress: Address, nodes: Set[Address], monitoredByNrOfMembers: Int) {
|
|
|
|
|
|
|
|
|
|
require(nodes contains selfAddress, s"nodes [${nodes.mkString(", ")}] must contain selfAddress [${selfAddress}]")
|
|
|
|
|
|
|
|
|
|
private val nodeRing: immutable.SortedSet[Address] = {
|
|
|
|
|
implicit val ringOrdering: Ordering[Address] = Ordering.fromLessThan[Address] { (a, b) ⇒
|
|
|
|
|
val ha = hashFor(a)
|
|
|
|
|
val hb = hashFor(b)
|
|
|
|
|
ha < hb || (ha == hb && Member.addressOrdering.compare(a, b) < 0)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
immutable.SortedSet() ++ nodes
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def hashFor(node: Address): Int = node match {
|
|
|
|
|
// cluster node identifier is the host and port of the address; protocol and system is assumed to be the same
|
|
|
|
|
case Address(_, _, Some(host), Some(port)) ⇒ MurmurHash.stringHash(s"${host}:${port}")
|
|
|
|
|
case _ ⇒ 0
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Receivers for `selfAddress`. Cached for subsequent access.
|
|
|
|
|
*/
|
|
|
|
|
lazy val myReceivers: immutable.Set[Address] = receivers(selfAddress)
|
|
|
|
|
/**
|
|
|
|
|
* Senders for `selfAddress`. Cached for subsequent access.
|
|
|
|
|
*/
|
|
|
|
|
lazy val mySenders: immutable.Set[Address] = senders(selfAddress)
|
|
|
|
|
|
|
|
|
|
private val useAllAsReceivers = monitoredByNrOfMembers >= (nodeRing.size - 1)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* The receivers to use from a specified sender.
|
|
|
|
|
*/
|
|
|
|
|
def receivers(sender: Address): immutable.Set[Address] =
|
|
|
|
|
if (useAllAsReceivers)
|
|
|
|
|
nodeRing - sender
|
|
|
|
|
else {
|
|
|
|
|
val slice = nodeRing.from(sender).tail.take(monitoredByNrOfMembers)
|
|
|
|
|
if (slice.size < monitoredByNrOfMembers)
|
|
|
|
|
(slice ++ nodeRing.take(monitoredByNrOfMembers - slice.size))
|
|
|
|
|
else slice
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* The expected senders for a specific receiver.
|
|
|
|
|
*/
|
|
|
|
|
def senders(receiver: Address): Set[Address] =
|
|
|
|
|
nodes filter { sender ⇒ receivers(sender) contains receiver }
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Add a node to the ring.
|
|
|
|
|
*/
|
|
|
|
|
def :+(node: Address): HeartbeatNodeRing = if (nodes contains node) this else copy(nodes = nodes + node)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Remove a node from the ring.
|
|
|
|
|
*/
|
|
|
|
|
def :-(node: Address): HeartbeatNodeRing = if (nodes contains node) copy(nodes = nodes - node) else this
|
|
|
|
|
|
|
|
|
|
}
|