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-07-06 17:04:04 +02:00
|
|
|
import akka.actor.{ ReceiveTimeout, ActorLogging, ActorRef, Address, Actor, RootActorPath, Props }
|
2012-07-05 13:55:08 +02:00
|
|
|
import java.security.MessageDigest
|
2012-07-06 17:04:04 +02:00
|
|
|
import akka.pattern.{ CircuitBreaker, CircuitBreakerOpenException }
|
|
|
|
|
import scala.concurrent.util.duration._
|
|
|
|
|
import scala.concurrent.util.Deadline
|
2012-10-01 10:02:48 +02:00
|
|
|
import scala.concurrent.util.FiniteDuration
|
|
|
|
|
import akka.cluster.ClusterEvent._
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Sent at regular intervals for failure detection.
|
|
|
|
|
*/
|
|
|
|
|
case class Heartbeat(from: Address) extends ClusterMessage
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* 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-09-06 21:48:40 +02:00
|
|
|
|
|
|
|
|
val failureDetector = Cluster(context.system).failureDetector
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
def receive = {
|
2012-09-06 21:48:40 +02:00
|
|
|
case Heartbeat(from) ⇒ failureDetector heartbeat from
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] object ClusterHeartbeatSender {
|
|
|
|
|
/**
|
2012-10-01 10:02:48 +02:00
|
|
|
* Command to [akka.cluster.ClusterHeartbeatSenderWorker]], which will send [[akka.cluster.Heartbeat]]
|
2012-07-05 13:55:08 +02:00
|
|
|
* to the other node.
|
|
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
|
|
|
|
case class SendHeartbeat(heartbeatMsg: Heartbeat, to: Address, deadline: Deadline)
|
2012-10-01 10:02:48 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Tell [akka.cluster.ClusterHeartbeatSender]] that this node has started joining of
|
|
|
|
|
* another node and heartbeats should be sent until it becomes member or deadline is overdue.
|
|
|
|
|
* Local only, no need to serialize.
|
|
|
|
|
*/
|
|
|
|
|
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
|
|
|
|
|
* other nodes. Netty blocks when sending to broken connections. This actor
|
|
|
|
|
* isolates sending to different nodes by using child workers for each target
|
|
|
|
|
* 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 10:02:48 +02:00
|
|
|
import Member.addressOrdering
|
|
|
|
|
import InternalClusterAction.HeartbeatTick
|
|
|
|
|
|
|
|
|
|
val cluster = Cluster(context.system)
|
|
|
|
|
import cluster.{ selfAddress, scheduler }
|
|
|
|
|
import cluster.settings._
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
|
|
|
|
|
val selfHeartbeat = Heartbeat(selfAddress)
|
|
|
|
|
|
|
|
|
|
var nodes: SortedSet[Address] = SortedSet.empty
|
|
|
|
|
var joinInProgress: Map[Address, Deadline] = Map.empty
|
|
|
|
|
|
|
|
|
|
// start periodic heartbeat to other nodes in cluster
|
|
|
|
|
val heartbeatTask =
|
|
|
|
|
FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(HeartbeatInterval).asInstanceOf[FiniteDuration], HeartbeatInterval) {
|
|
|
|
|
self ! HeartbeatTick
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
cluster.subscribe(self, classOf[MemberEvent])
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
val digester = MessageDigest.getInstance("MD5")
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Child name is MD5 hash of the address.
|
|
|
|
|
* FIXME Change to URLEncode when ticket #2123 has been fixed
|
|
|
|
|
*/
|
|
|
|
|
def encodeChildName(name: String): String = {
|
|
|
|
|
digester update name.getBytes("UTF-8")
|
|
|
|
|
digester.digest.map { h ⇒ "%02x".format(0xFF & h) }.mkString
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receive = {
|
2012-10-01 10:02:48 +02:00
|
|
|
case state: CurrentClusterState ⇒ init(state)
|
|
|
|
|
case MemberUnreachable(m) ⇒ removeMember(m)
|
|
|
|
|
case MemberRemoved(m) ⇒ removeMember(m)
|
|
|
|
|
case e: MemberEvent ⇒ addMember(e.member)
|
|
|
|
|
case JoinInProgress(a, d) ⇒ joinInProgress += (a -> d)
|
|
|
|
|
case HeartbeatTick ⇒ heartbeat()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def init(state: CurrentClusterState): Unit = {
|
|
|
|
|
nodes = state.members.map(_.address)
|
|
|
|
|
joinInProgress --= nodes
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def addMember(m: Member): Unit = {
|
|
|
|
|
nodes += m.address
|
|
|
|
|
joinInProgress -= m.address
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def removeMember(m: Member): Unit = {
|
|
|
|
|
nodes -= m.address
|
|
|
|
|
joinInProgress -= m.address
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def heartbeat(): Unit = {
|
|
|
|
|
removeOverdueJoinInProgress()
|
|
|
|
|
|
|
|
|
|
val beatTo = nodes ++ joinInProgress.keys
|
|
|
|
|
|
|
|
|
|
val deadline = Deadline.now + HeartbeatInterval
|
|
|
|
|
for (to ← beatTo; if to != selfAddress) {
|
2012-07-05 13:55:08 +02:00
|
|
|
val workerName = encodeChildName(to.toString)
|
|
|
|
|
val worker = context.actorFor(workerName) match {
|
|
|
|
|
case notFound if notFound.isTerminated ⇒
|
2012-09-06 21:48:40 +02:00
|
|
|
context.actorOf(Props(new ClusterHeartbeatSenderWorker(clusterHeartbeatConnectionFor(to))), workerName)
|
2012-07-05 13:55:08 +02:00
|
|
|
case child ⇒ child
|
|
|
|
|
}
|
2012-10-01 10:02:48 +02:00
|
|
|
worker ! SendHeartbeat(selfHeartbeat, to, deadline)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Removes overdue joinInProgress from State.
|
|
|
|
|
*/
|
|
|
|
|
def removeOverdueJoinInProgress(): Unit = {
|
|
|
|
|
joinInProgress --= joinInProgress collect { case (address, deadline) if (nodes contains address) || deadline.isOverdue ⇒ address }
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Responsible for sending [[akka.cluster.Heartbeat]] to one specific address.
|
|
|
|
|
*
|
|
|
|
|
* Netty blocks when sending to broken connections, and this actor uses
|
|
|
|
|
* a configurable circuit breaker to reduce connect attempts to broken
|
|
|
|
|
* connections.
|
|
|
|
|
*
|
|
|
|
|
* @see ClusterHeartbeatSender
|
|
|
|
|
*/
|
2012-09-06 21:48:40 +02:00
|
|
|
private[cluster] final class ClusterHeartbeatSenderWorker(toRef: ActorRef)
|
2012-07-05 13:55:08 +02:00
|
|
|
extends Actor with ActorLogging {
|
|
|
|
|
|
|
|
|
|
import ClusterHeartbeatSender._
|
|
|
|
|
|
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
|
|
|
|
2012-07-06 15:21:11 +02:00
|
|
|
// make sure it will cleanup when not used any more
|
2012-07-05 13:55:08 +02:00
|
|
|
context.setReceiveTimeout(30 seconds)
|
|
|
|
|
|
|
|
|
|
def receive = {
|
|
|
|
|
case SendHeartbeat(heartbeatMsg, _, deadline) ⇒
|
|
|
|
|
if (!deadline.isOverdue) {
|
|
|
|
|
// the CircuitBreaker will measure elapsed time and open if too many long calls
|
|
|
|
|
try breaker.withSyncCircuitBreaker {
|
|
|
|
|
log.debug("Cluster Node [{}] - Heartbeat to [{}]", heartbeatMsg.from, toRef)
|
|
|
|
|
toRef ! heartbeatMsg
|
|
|
|
|
if (deadline.isOverdue) log.debug("Sending heartbeat to [{}] took longer than expected", toRef)
|
|
|
|
|
} catch { case e: CircuitBreakerOpenException ⇒ /* skip sending heartbeat to broken connection */ }
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case ReceiveTimeout ⇒ context.stop(self) // cleanup when not used
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|