2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.cluster
|
|
|
|
|
|
2012-07-06 17:04:04 +02:00
|
|
|
import language.postfixOps
|
|
|
|
|
|
2012-10-01 10:02:48 +02:00
|
|
|
import scala.collection.immutable.SortedSet
|
2012-10-01 14:12:20 +02:00
|
|
|
import scala.annotation.tailrec
|
2012-09-21 14:50:06 +02:00
|
|
|
import scala.concurrent.duration._
|
2012-10-01 11:15:29 +02:00
|
|
|
import java.net.URLEncoder
|
2012-10-01 14:12:20 +02:00
|
|
|
import akka.actor.{ ActorLogging, ActorRef, Address, Actor, RootActorPath, PoisonPill, Props }
|
|
|
|
|
import akka.pattern.{ CircuitBreaker, CircuitBreakerOpenException }
|
|
|
|
|
import akka.cluster.ClusterEvent._
|
|
|
|
|
import akka.routing.ConsistentHash
|
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 {
|
|
|
|
|
/**
|
2012-10-01 10:02:48 +02:00
|
|
|
* Tell [akka.cluster.ClusterHeartbeatSender]] that this node has started joining of
|
2012-10-01 14:12:20 +02:00
|
|
|
* another node and heartbeats should be sent unconditionally until it becomes
|
|
|
|
|
* member or deadline is overdue. This is done to be able to detect immediate death
|
|
|
|
|
* of the joining node.
|
2012-07-05 13:55:08 +02:00
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
2012-10-01 10:02:48 +02:00
|
|
|
case class JoinInProgress(address: Address, deadline: Deadline)
|
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.
|
|
|
|
|
*
|
|
|
|
|
* Netty blocks when sending to broken connections. This actor
|
|
|
|
|
* isolates sending to different nodes by using child actors for each target
|
2012-07-05 13:55:08 +02:00
|
|
|
* address and thereby reduce the risk of irregular heartbeats to healty
|
|
|
|
|
* nodes due to broken connections to other nodes.
|
|
|
|
|
*/
|
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 ClusterHeartbeatSenderConnection._
|
|
|
|
|
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._
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
|
|
|
|
|
val selfHeartbeat = Heartbeat(selfAddress)
|
2012-10-01 14:12:20 +02:00
|
|
|
val selfEndHeartbeat = EndHeartbeat(selfAddress)
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
var state = ClusterHeartbeatSenderState.empty(ConsistentHash(Seq.empty[Address], HeartbeatConsistentHashingVirtualNodesFactor),
|
|
|
|
|
selfAddress.toString, 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-10-09 17:54:54 +02:00
|
|
|
override def preStart(): Unit = cluster.subscribe(self, classOf[MemberEvent])
|
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.
|
|
|
|
|
*/
|
|
|
|
|
def clusterHeartbeatConnectionFor(address: Address): ActorRef =
|
2012-10-01 10:02:48 +02:00
|
|
|
context.actorFor(RootActorPath(address) / "system" / "cluster" / "heartbeatReceiver")
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
def receive = {
|
2012-10-10 15:23:18 +02:00
|
|
|
case HeartbeatTick ⇒ heartbeat()
|
|
|
|
|
case s: CurrentClusterState ⇒ reset(s)
|
|
|
|
|
case MemberUnreachable(m) ⇒ removeMember(m)
|
|
|
|
|
case MemberRemoved(m) ⇒ removeMember(m)
|
|
|
|
|
case e: MemberEvent ⇒ addMember(e.member)
|
|
|
|
|
case JoinInProgress(a, d) ⇒ addJoinInProgress(a, d)
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
def reset(snapshot: CurrentClusterState): Unit =
|
|
|
|
|
state = state.reset(snapshot.members.collect { case m if m.address != selfAddress ⇒ m.address })
|
2012-10-01 14:12:20 +02:00
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
def addMember(m: Member): Unit = if (m.address != selfAddress)
|
|
|
|
|
state = state addMember m.address
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
def removeMember(m: Member): Unit = if (m.address != selfAddress)
|
|
|
|
|
state = state removeMember m.address
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2012-10-10 15:23:18 +02:00
|
|
|
def addJoinInProgress(address: Address, deadline: Deadline): Unit = if (address != selfAddress)
|
|
|
|
|
state = state.addJoinInProgress(address, deadline)
|
2012-10-01 10:02:48 +02:00
|
|
|
|
|
|
|
|
def heartbeat(): Unit = {
|
2012-10-10 15:23:18 +02:00
|
|
|
state = state.removeOverdueJoinInProgress()
|
2012-10-01 10:02:48 +02:00
|
|
|
|
2012-10-01 14:12:20 +02:00
|
|
|
def connection(to: Address): ActorRef = {
|
|
|
|
|
// URL encoded target address as child actor name
|
|
|
|
|
val connectionName = URLEncoder.encode(to.toString, "UTF-8")
|
|
|
|
|
context.actorFor(connectionName) match {
|
2012-07-05 13:55:08 +02:00
|
|
|
case notFound if notFound.isTerminated ⇒
|
2012-10-01 14:12:20 +02:00
|
|
|
context.actorOf(Props(new ClusterHeartbeatSenderConnection(clusterHeartbeatConnectionFor(to))), connectionName)
|
2012-07-05 13:55:08 +02:00
|
|
|
case child ⇒ child
|
|
|
|
|
}
|
2012-10-01 10:02:48 +02:00
|
|
|
}
|
2012-10-01 14:12:20 +02:00
|
|
|
|
|
|
|
|
val deadline = Deadline.now + HeartbeatInterval
|
2012-10-10 15:23:18 +02:00
|
|
|
state.active foreach { to ⇒ connection(to) ! SendHeartbeat(selfHeartbeat, to, deadline) }
|
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) {
|
2012-10-01 14:12:20 +02:00
|
|
|
val c = connection(to)
|
|
|
|
|
c ! SendEndHeartbeat(selfEndHeartbeat, to)
|
|
|
|
|
if (count == NumberOfEndHeartbeats) {
|
2012-10-10 15:23:18 +02:00
|
|
|
state = state.removeEnding(to)
|
2012-10-01 14:12:20 +02:00
|
|
|
c ! PoisonPill
|
2012-10-10 15:23:18 +02:00
|
|
|
} else
|
|
|
|
|
state = state.increaseEndingCount(to)
|
2012-10-01 14:12:20 +02:00
|
|
|
}
|
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
|
|
|
|
|
*/
|
|
|
|
|
def empty(consistentHash: ConsistentHash[Address], selfAddressStr: String,
|
|
|
|
|
monitoredByNrOfMembers: Int): ClusterHeartbeatSenderState =
|
|
|
|
|
ClusterHeartbeatSenderState(consistentHash, selfAddressStr, monitoredByNrOfMembers)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* 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,
|
|
|
|
|
consistentHash: ConsistentHash[Address],
|
|
|
|
|
all: Set[Address]): ClusterHeartbeatSenderState = {
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Select a few peers that heartbeats will be sent to, i.e. that will
|
|
|
|
|
* monitor this node. Try to send heartbeats to same nodes as much
|
|
|
|
|
* as possible, but re-balance with consistent hashing algorithm when
|
|
|
|
|
* new members are added or removed.
|
|
|
|
|
*/
|
|
|
|
|
def selectPeers: Set[Address] = {
|
|
|
|
|
val allSize = all.size
|
|
|
|
|
val nrOfPeers = math.min(allSize, old.monitoredByNrOfMembers)
|
|
|
|
|
// try more if consistentHash results in same node as already selected
|
|
|
|
|
val attemptLimit = nrOfPeers * 2
|
|
|
|
|
@tailrec def select(acc: Set[Address], n: Int): Set[Address] = {
|
|
|
|
|
if (acc.size == nrOfPeers || n == attemptLimit) acc
|
|
|
|
|
else select(acc + consistentHash.nodeFor(old.selfAddressStr + n), n + 1)
|
|
|
|
|
}
|
|
|
|
|
if (nrOfPeers >= allSize) all
|
|
|
|
|
else select(Set.empty[Address], 0)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val curr = selectPeers
|
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
|
2012-10-10 18:13:08 +02:00
|
|
|
old.copy(consistentHash = consistentHash, all = all, current = curr, ending = end)
|
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 (
|
|
|
|
|
consistentHash: ConsistentHash[Address],
|
|
|
|
|
selfAddressStr: String,
|
|
|
|
|
monitoredByNrOfMembers: Int,
|
|
|
|
|
all: Set[Address] = Set.empty,
|
|
|
|
|
current: Set[Address] = Set.empty,
|
|
|
|
|
ending: Map[Address, Int] = Map.empty,
|
|
|
|
|
joinInProgress: Map[Address, Deadline] = Map.empty) {
|
|
|
|
|
|
|
|
|
|
// FIXME can be disabled as optimization
|
|
|
|
|
assertInvariants
|
|
|
|
|
|
|
|
|
|
private def assertInvariants: Unit = {
|
|
|
|
|
val currentAndEnding = current.intersect(ending.keySet)
|
|
|
|
|
require(currentAndEnding.isEmpty,
|
|
|
|
|
"Same nodes in current and ending not allowed, got [%s]" format currentAndEnding)
|
|
|
|
|
val joinInProgressAndAll = joinInProgress.keySet.intersect(all)
|
|
|
|
|
require(joinInProgressAndAll.isEmpty,
|
|
|
|
|
"Same nodes in joinInProgress and all not allowed, got [%s]" format joinInProgressAndAll)
|
|
|
|
|
val currentNotInAll = current -- all
|
|
|
|
|
require(currentNotInAll.isEmpty,
|
|
|
|
|
"Nodes in current but not in all not allowed, got [%s]" format currentNotInAll)
|
|
|
|
|
require(all.isEmpty == consistentHash.isEmpty, "ConsistentHash doesn't correspond to all nodes [%s]"
|
|
|
|
|
format all)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val active: Set[Address] = current ++ joinInProgress.keySet
|
|
|
|
|
|
|
|
|
|
def reset(nodes: Set[Address]): ClusterHeartbeatSenderState =
|
2012-10-10 18:13:08 +02:00
|
|
|
ClusterHeartbeatSenderState(nodes.foldLeft(this) { _ removeJoinInProgress _ },
|
|
|
|
|
consistentHash = ConsistentHash(nodes, consistentHash.virtualNodesFactor),
|
2012-10-10 15:23:18 +02:00
|
|
|
all = nodes)
|
|
|
|
|
|
|
|
|
|
def addMember(a: Address): ClusterHeartbeatSenderState =
|
2012-10-10 18:13:08 +02:00
|
|
|
ClusterHeartbeatSenderState(removeJoinInProgress(a), all = all + a, consistentHash = consistentHash :+ a)
|
2012-10-10 15:23:18 +02:00
|
|
|
|
|
|
|
|
def removeMember(a: Address): ClusterHeartbeatSenderState =
|
2012-10-10 18:13:08 +02:00
|
|
|
ClusterHeartbeatSenderState(removeJoinInProgress(a), all = all - a, consistentHash = consistentHash :- a)
|
2012-10-10 15:23:18 +02:00
|
|
|
|
2012-10-10 18:13:08 +02:00
|
|
|
private def removeJoinInProgress(address: Address): ClusterHeartbeatSenderState = {
|
2012-10-10 15:23:18 +02:00
|
|
|
if (joinInProgress contains address)
|
|
|
|
|
copy(joinInProgress = joinInProgress - address, ending = ending + (address -> 0))
|
|
|
|
|
else this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def addJoinInProgress(address: Address, deadline: Deadline): ClusterHeartbeatSenderState = {
|
|
|
|
|
if (all contains address) this
|
2012-10-10 18:13:08 +02:00
|
|
|
else copy(joinInProgress = joinInProgress + (address -> deadline), ending = ending - address)
|
2012-10-01 14:12:20 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Cleanup overdue joinInProgress, in case a joining node never
|
|
|
|
|
* became member, for some reason.
|
2012-10-01 10:02:48 +02:00
|
|
|
*/
|
2012-10-10 15:23:18 +02:00
|
|
|
def removeOverdueJoinInProgress(): ClusterHeartbeatSenderState = {
|
2012-10-01 14:12:20 +02:00
|
|
|
val overdue = joinInProgress collect { case (address, deadline) if deadline.isOverdue ⇒ address }
|
2012-10-10 15:23:18 +02:00
|
|
|
if (overdue.isEmpty) this
|
|
|
|
|
else
|
|
|
|
|
copy(ending = ending ++ overdue.map(_ -> 0), joinInProgress = joinInProgress -- 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
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2012-10-01 14:12:20 +02:00
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] object ClusterHeartbeatSenderConnection {
|
|
|
|
|
import ClusterHeartbeatReceiver._
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Command to [akka.cluster.ClusterHeartbeatSenderConnection]], which will send
|
|
|
|
|
* [[akka.cluster.ClusterHeartbeatReceiver.Heartbeat]] to the other node.
|
|
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
|
|
|
|
case class SendHeartbeat(heartbeatMsg: Heartbeat, to: Address, deadline: Deadline)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Command to [akka.cluster.ClusterHeartbeatSenderConnection]], which will send
|
|
|
|
|
* [[akka.cluster.ClusterHeartbeatReceiver.EndHeartbeat]] to the other node.
|
|
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
|
|
|
|
case class SendEndHeartbeat(endHeartbeatMsg: EndHeartbeat, to: Address)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Responsible for sending [[akka.cluster.ClusterHeartbeatReceiver.Heartbeat]]
|
|
|
|
|
* and [[akka.cluster.ClusterHeartbeatReceiver.EndHeartbeat]] to one specific address.
|
2012-07-05 13:55:08 +02:00
|
|
|
*
|
2012-10-09 17:54:54 +02:00
|
|
|
* This actor exists only because Netty blocks when sending to broken connections,
|
|
|
|
|
* and this actor uses a configurable circuit breaker to reduce connect attempts to broken
|
2012-07-05 13:55:08 +02:00
|
|
|
* connections.
|
|
|
|
|
*
|
2012-10-09 17:54:54 +02:00
|
|
|
* @see akka.cluster.ClusterHeartbeatSender
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2012-10-01 14:12:20 +02:00
|
|
|
private[cluster] final class ClusterHeartbeatSenderConnection(toRef: ActorRef)
|
2012-07-05 13:55:08 +02:00
|
|
|
extends Actor with ActorLogging {
|
|
|
|
|
|
2012-10-01 14:12:20 +02:00
|
|
|
import ClusterHeartbeatSenderConnection._
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2012-09-06 21:48:40 +02:00
|
|
|
val breaker = {
|
|
|
|
|
val cbSettings = Cluster(context.system).settings.SendCircuitBreakerSettings
|
|
|
|
|
CircuitBreaker(context.system.scheduler,
|
|
|
|
|
cbSettings.maxFailures, cbSettings.callTimeout, cbSettings.resetTimeout).
|
|
|
|
|
onHalfOpen(log.debug("CircuitBreaker Half-Open for: [{}]", toRef)).
|
|
|
|
|
onOpen(log.debug("CircuitBreaker Open for [{}]", toRef)).
|
|
|
|
|
onClose(log.debug("CircuitBreaker Closed for [{}]", toRef))
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
def receive = {
|
|
|
|
|
case SendHeartbeat(heartbeatMsg, _, deadline) ⇒
|
|
|
|
|
if (!deadline.isOverdue) {
|
2012-10-01 14:12:20 +02:00
|
|
|
log.debug("Cluster Node [{}] - Heartbeat to [{}]", heartbeatMsg.from, toRef)
|
2012-10-09 17:54:54 +02:00
|
|
|
// Netty blocks when sending to broken connections, the CircuitBreaker will
|
|
|
|
|
// measure elapsed time and open if too many long calls
|
2012-07-05 13:55:08 +02:00
|
|
|
try breaker.withSyncCircuitBreaker {
|
|
|
|
|
toRef ! heartbeatMsg
|
|
|
|
|
} catch { case e: CircuitBreakerOpenException ⇒ /* skip sending heartbeat to broken connection */ }
|
|
|
|
|
}
|
2012-10-01 14:12:20 +02:00
|
|
|
if (deadline.isOverdue) log.debug("Sending heartbeat to [{}] took longer than expected", toRef)
|
|
|
|
|
case SendEndHeartbeat(endHeartbeatMsg, _) ⇒
|
|
|
|
|
log.debug("Cluster Node [{}] - EndHeartbeat to [{}]", endHeartbeatMsg.from, toRef)
|
|
|
|
|
toRef ! endHeartbeatMsg
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
2012-09-21 14:50:06 +02:00
|
|
|
}
|