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-12-10 08:46:25 +01:00
import language.existentials
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._
2012-07-06 17:04:04 +02:00
import scala.concurrent.forkjoin.ThreadLocalRandom
2012-12-10 08:46:25 +01:00
import scala.util.control.NonFatal
2013-02-12 21:45:41 +01:00
import java.util.UUID
2013-02-11 10:40:01 +01:00
import akka.actor. { Actor , ActorLogging , ActorRef , Address , Cancellable , Props , PoisonPill , ReceiveTimeout , RootActorPath , Scheduler }
import akka.actor.OneForOneStrategy
import akka.actor.SupervisorStrategy.Stop
2012-08-14 10:58:30 +02:00
import akka.cluster.MemberStatus._
import akka.cluster.ClusterEvent._
2013-03-26 18:17:50 +01:00
import akka.actor.ActorSelection
2012-07-05 13:55:08 +02:00
/* *
* Base trait for all cluster messages . All ClusterMessage 's are serializable .
*/
trait ClusterMessage extends Serializable
/* *
* Cluster commands sent by the USER .
*/
object ClusterUserAction {
/* *
* Command to join the cluster . Sent when a node ( represented by 'address' )
* wants to join another node ( the receiver ) .
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2013-03-14 20:32:43 +01:00
case class Join ( address : Address , roles : Set [ String ] ) extends ClusterMessage
2012-07-05 13:55:08 +02:00
/* *
* Command to leave the cluster .
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
case class Leave ( address : Address ) extends ClusterMessage
/* *
* Command to mark node as temporary down .
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
case class Down ( address : Address ) extends ClusterMessage
}
/* *
* INTERNAL API
*/
private [ cluster ] object InternalClusterAction {
/* *
* Command to initiate join another node ( represented by 'address' ) .
* Join will be sent to the other node .
*/
2013-04-04 17:56:29 +02:00
case class JoinTo ( address : Address )
2012-07-05 13:55:08 +02:00
2012-09-06 21:48:40 +02:00
/* *
* Command to initiate the process to join the specified
* seed nodes .
*/
2012-11-15 12:33:11 +01:00
case class JoinSeedNodes ( seedNodes : immutable.IndexedSeq [ Address ] )
2012-09-06 21:48:40 +02:00
2012-07-05 13:55:08 +02:00
/* *
* Start message of the process to join one of the seed nodes .
* The node sends `InitJoin` to all seed nodes , which replies
* with `InitJoinAck` . The first reply is used others are discarded .
* The node sends `Join` command to the seed node that replied first .
2013-02-17 17:35:43 +01:00
* If a node is uninitialized it will reply to `InitJoin` with
* `InitJoinNack` .
2012-07-05 13:55:08 +02:00
*/
2013-04-04 17:56:29 +02:00
case object JoinSeedNode
2012-07-05 13:55:08 +02:00
/* *
* @see JoinSeedNode
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
case object InitJoin extends ClusterMessage
/* *
* @see JoinSeedNode
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
case class InitJoinAck ( address : Address ) extends ClusterMessage
2013-02-17 17:35:43 +01:00
/* *
* @see JoinSeedNode
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2013-02-17 17:35:43 +01:00
case class InitJoinNack ( address : Address ) extends ClusterMessage
2012-08-14 13:55:22 +02:00
/* *
* Marker interface for periodic tick messages
*/
2012-08-14 17:30:49 +02:00
sealed trait Tick
2012-07-05 13:55:08 +02:00
2012-08-14 13:55:22 +02:00
case object GossipTick extends Tick
2012-07-05 13:55:08 +02:00
2012-08-14 13:55:22 +02:00
case object HeartbeatTick extends Tick
2012-07-05 13:55:08 +02:00
2012-08-14 13:55:22 +02:00
case object ReapUnreachableTick extends Tick
2012-07-05 13:55:08 +02:00
2012-09-24 13:07:11 -06:00
case object MetricsTick extends Tick
2012-08-14 13:55:22 +02:00
case object LeaderActionsTick extends Tick
2012-08-16 18:54:10 +02:00
case object PublishStatsTick extends Tick
2012-07-05 13:55:08 +02:00
case class SendGossipTo ( address : Address )
case object GetClusterCoreRef
2013-02-11 10:40:01 +01:00
case class PublisherCreated ( publisher : ActorRef )
2012-12-10 08:46:25 +01:00
/* *
* Comand to [ [ akka . cluster . ClusterDaemon ] ] to create a
* [ [ akka . cluster . OnMemberUpListener ] ] .
*/
case class AddOnMemberUpListener ( callback : Runnable )
2012-08-19 20:15:22 +02:00
sealed trait SubscriptionMessage
case class Subscribe ( subscriber : ActorRef , to : Class [ _ ] ) extends SubscriptionMessage
2012-09-28 13:09:36 +02:00
case class Unsubscribe ( subscriber : ActorRef , to : Option [ Class [ _ ] ] ) extends SubscriptionMessage
2012-09-12 09:23:02 +02:00
/* *
* @param receiver if `receiver` is defined the event will only be sent to that
* actor , otherwise it will be sent to all subscribers via the `eventStream` .
*/
case class PublishCurrentClusterState ( receiver : Option [ ActorRef ] ) extends SubscriptionMessage
2012-08-19 20:15:22 +02:00
2012-11-27 18:07:37 +01:00
sealed trait PublishMessage
case class PublishChanges ( newGossip : Gossip ) extends PublishMessage
case class PublishEvent ( event : ClusterDomainEvent ) extends PublishMessage
case object PublishStart extends PublishMessage
2012-07-05 13:55:08 +02:00
}
/* *
* INTERNAL API .
*
* Cluster commands sent by the LEADER .
*/
private [ cluster ] object ClusterLeaderAction {
/* *
* Command to mark a node to be removed from the cluster immediately .
* Can only be sent by the leader .
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
case class Exit ( address : Address ) extends ClusterMessage
/* *
* Command to remove a node from the cluster immediately .
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
case class Remove ( address : Address ) extends ClusterMessage
}
/* *
* INTERNAL API .
*
* Supervisor managing the different Cluster daemons .
*/
2012-09-07 17:42:15 +02:00
private [ cluster ] final class ClusterDaemon ( settings : ClusterSettings ) extends Actor with ActorLogging {
2013-02-11 10:40:01 +01:00
import InternalClusterAction._
2012-09-06 21:48:40 +02:00
// Important - don't use Cluster(context.system) here because that would
// cause deadlock. The Cluster extension is currently being created and is waiting
// for response from GetClusterCoreRef in its constructor.
2013-02-11 10:40:01 +01:00
val coreSupervisor = context . actorOf ( Props [ ClusterCoreSupervisor ] .
2012-09-06 21:48:40 +02:00
withDispatcher ( context . props . dispatcher ) , name = "core" )
2012-10-01 10:02:48 +02:00
context . actorOf ( Props [ ClusterHeartbeatReceiver ] .
withDispatcher ( context . props . dispatcher ) , name = "heartbeatReceiver" )
2012-07-05 13:55:08 +02:00
def receive = {
2013-02-11 10:40:01 +01:00
case msg @ GetClusterCoreRef ⇒ coreSupervisor forward msg
case AddOnMemberUpListener ( code ) ⇒
2012-12-10 08:46:25 +01:00
context . actorOf ( Props ( new OnMemberUpListener ( code ) ) )
2013-02-11 10:40:01 +01:00
case PublisherCreated ( publisher ) ⇒
if ( settings . MetricsEnabled ) {
// metrics must be started after core/publisher to be able
// to inject the publisher ref to the ClusterMetricsCollector
context . actorOf ( Props ( new ClusterMetricsCollector ( publisher ) ) .
withDispatcher ( context . props . dispatcher ) , name = "metrics" )
}
2012-07-05 13:55:08 +02:00
}
}
2013-02-11 10:40:01 +01:00
/* *
* INTERNAL API .
*
* ClusterCoreDaemon and ClusterDomainEventPublisher can 't be restarted because the state
* would be obsolete . Shutdown the member if any those actors crashed .
*/
private [ cluster ] final class ClusterCoreSupervisor extends Actor with ActorLogging {
import InternalClusterAction._
val publisher = context . actorOf ( Props [ ClusterDomainEventPublisher ] .
withDispatcher ( context . props . dispatcher ) , name = "publisher" )
val coreDaemon = context . watch ( context . actorOf ( Props ( new ClusterCoreDaemon ( publisher ) ) .
withDispatcher ( context . props . dispatcher ) , name = "daemon" ) )
context . parent ! PublisherCreated ( publisher )
override val supervisorStrategy =
OneForOneStrategy ( ) {
case NonFatal ( e ) ⇒
log . error ( e , "Cluster node [{}] crashed, [{}] - shutting down..." , Cluster ( context . system ) . selfAddress , e . getMessage )
self ! PoisonPill
Stop
}
override def postStop ( ) : Unit = Cluster ( context . system ) . shutdown ( )
def receive = {
case InternalClusterAction . GetClusterCoreRef ⇒ sender ! coreDaemon
}
}
2012-07-05 13:55:08 +02:00
/* *
* INTERNAL API .
*/
2012-10-02 16:41:03 +02:00
private [ cluster ] final class ClusterCoreDaemon ( publisher : ActorRef ) extends Actor with ActorLogging {
2012-07-05 13:55:08 +02:00
import ClusterLeaderAction._
import InternalClusterAction._
2012-09-06 21:48:40 +02:00
val cluster = Cluster ( context . system )
2012-09-07 17:42:15 +02:00
import cluster. { selfAddress , scheduler , failureDetector }
2012-09-06 21:48:40 +02:00
import cluster.settings._
2012-07-05 13:55:08 +02:00
2013-02-12 21:45:41 +01:00
// FIXME the UUID should not be needed when Address contains uid, ticket #2788
val vclockNode = VectorClock . Node ( selfAddress . toString + "-" + UUID . randomUUID ( ) )
2012-07-05 13:55:08 +02:00
// note that self is not initially member,
// and the Gossip is not versioned for this 'Node' yet
2013-01-04 16:39:48 +01:00
var latestGossip : Gossip = Gossip . empty
2012-07-05 13:55:08 +02:00
var stats = ClusterStats ( )
2013-02-17 17:35:43 +01:00
var seedNodeProcess : Option [ ActorRef ] = None
2013-03-20 10:32:18 +01:00
var tryingToJoinWith : Option [ Address ] = None
2013-01-25 15:03:52 +01:00
/* *
* Looks up and returns the remote cluster command connection for the specific address .
*/
2013-03-26 18:17:50 +01:00
private def clusterCore ( address : Address ) : ActorSelection =
context . actorSelection ( RootActorPath ( address ) / "system" / "cluster" / "core" / "daemon" )
2013-01-25 15:03:52 +01:00
2012-10-01 10:02:48 +02:00
val heartbeatSender = context . actorOf ( Props [ ClusterHeartbeatSender ] .
withDispatcher ( UseDispatcher ) , name = "heartbeatSender" )
2012-07-05 13:55:08 +02:00
2012-08-08 15:57:30 +02:00
import context.dispatcher
2012-07-05 13:55:08 +02:00
// start periodic gossip to random nodes in cluster
2012-09-21 14:50:06 +02:00
val gossipTask = scheduler . schedule ( PeriodicTasksInitialDelay . max ( GossipInterval ) ,
2012-10-08 12:17:40 +02:00
GossipInterval , self , GossipTick )
2012-07-05 13:55:08 +02:00
// start periodic cluster failure detector reaping (moving nodes condemned by the failure detector to unreachable list)
2012-09-21 14:50:06 +02:00
val failureDetectorReaperTask = scheduler . schedule ( PeriodicTasksInitialDelay . max ( UnreachableNodesReaperInterval ) ,
2012-10-08 12:17:40 +02:00
UnreachableNodesReaperInterval , self , ReapUnreachableTick )
2012-07-05 13:55:08 +02:00
// start periodic leader action management (only applies for the current leader)
2012-09-21 14:50:06 +02:00
val leaderActionsTask = scheduler . schedule ( PeriodicTasksInitialDelay . max ( LeaderActionsInterval ) ,
2012-10-08 12:17:40 +02:00
LeaderActionsInterval , self , LeaderActionsTick )
2012-07-05 13:55:08 +02:00
2012-09-24 13:07:11 -06:00
// start periodic publish of current stats
2012-10-08 12:17:40 +02:00
val publishStatsTask : Option [ Cancellable ] =
2012-08-15 16:47:34 +02:00
if ( PublishStatsInterval == Duration . Zero ) None
2012-09-21 14:50:06 +02:00
else Some ( scheduler . schedule ( PeriodicTasksInitialDelay . max ( PublishStatsInterval ) ,
2012-10-08 12:17:40 +02:00
PublishStatsInterval , self , PublishStatsTick ) )
2012-07-05 13:55:08 +02:00
override def preStart ( ) : Unit = {
2012-09-06 21:48:40 +02:00
if ( AutoJoin ) self ! JoinSeedNodes ( SeedNodes )
2012-07-05 13:55:08 +02:00
}
override def postStop ( ) : Unit = {
gossipTask . cancel ( )
failureDetectorReaperTask . cancel ( )
leaderActionsTask . cancel ( )
2012-09-24 13:07:11 -06:00
publishStatsTask foreach { _ . cancel ( ) }
2012-07-05 13:55:08 +02:00
}
2012-08-14 13:55:22 +02:00
def uninitialized : Actor . Receive = {
2013-02-17 17:35:43 +01:00
case InitJoin ⇒ sender ! InitJoinNack ( selfAddress )
2012-08-19 20:15:22 +02:00
case JoinTo ( address ) ⇒ join ( address )
2012-09-06 21:48:40 +02:00
case JoinSeedNodes ( seedNodes ) ⇒ joinSeedNodes ( seedNodes )
2012-08-19 20:15:22 +02:00
case msg : SubscriptionMessage ⇒ publisher forward msg
case _ : Tick ⇒ // ignore periodic tasks until initialized
2012-08-14 13:55:22 +02:00
}
def initialized : Actor . Receive = {
2013-03-14 20:32:43 +01:00
case msg : GossipEnvelope ⇒ receiveGossip ( msg )
case GossipTick ⇒ gossip ( )
case ReapUnreachableTick ⇒ reapUnreachableMembers ( )
case LeaderActionsTick ⇒ leaderActions ( )
case PublishStatsTick ⇒ publishInternalStats ( )
case InitJoin ⇒ initJoin ( )
case JoinTo ( address ) ⇒ join ( address )
case ClusterUserAction . Join ( address , roles ) ⇒ joining ( address , roles )
case ClusterUserAction . Down ( address ) ⇒ downing ( address )
case ClusterUserAction . Leave ( address ) ⇒ leaving ( address )
case Exit ( address ) ⇒ exiting ( address )
case Remove ( address ) ⇒ removing ( address )
case SendGossipTo ( address ) ⇒ gossipTo ( address )
case msg : SubscriptionMessage ⇒ publisher forward msg
2012-07-05 13:55:08 +02:00
}
2012-08-19 20:15:22 +02:00
def removed : Actor . Receive = {
case msg : SubscriptionMessage ⇒ publisher forward msg
case _ : Tick ⇒ // ignore periodic tasks
}
2012-08-14 13:55:22 +02:00
def receive = uninitialized
2012-07-05 13:55:08 +02:00
def initJoin ( ) : Unit = sender ! InitJoinAck ( selfAddress )
2012-11-15 12:33:11 +01:00
def joinSeedNodes ( seedNodes : immutable.IndexedSeq [ Address ] ) : Unit = {
2013-02-17 17:35:43 +01:00
require ( seedNodeProcess . isEmpty , "Join seed nodes is already in progress" )
seedNodeProcess =
if ( seedNodes . isEmpty || seedNodes == immutable . IndexedSeq ( selfAddress ) ) {
self ! JoinTo ( selfAddress )
None
} else if ( seedNodes . head == selfAddress ) {
Some ( context . actorOf ( Props ( new FirstSeedNodeProcess ( seedNodes ) ) .
withDispatcher ( UseDispatcher ) , name = "firstSeedNodeProcess" ) )
} else {
Some ( context . actorOf ( Props ( new JoinSeedNodeProcess ( seedNodes ) ) .
withDispatcher ( UseDispatcher ) , name = "joinSeedNodeProcess" ) )
}
2012-09-06 21:48:40 +02:00
}
2012-07-05 13:55:08 +02:00
/* *
* Try to join this cluster node with the node specified by 'address' .
* A 'Join( thisNodeAddress ) ' command is sent to the node to join .
*/
def join ( address : Address ) : Unit = {
2013-01-15 09:35:07 +01:00
if ( address . protocol != selfAddress . protocol )
2013-02-17 17:35:43 +01:00
log . warning ( "Trying to join member with wrong protocol, but was ignored, expected [{}] but was [{}]" ,
2013-01-15 09:35:07 +01:00
selfAddress . protocol , address . protocol )
else if ( address . system != selfAddress . system )
2013-02-17 17:35:43 +01:00
log . warning ( "Trying to join member with wrong ActorSystem name, but was ignored, expected [{}] but was [{}]" ,
2013-01-15 09:35:07 +01:00
selfAddress . system , address . system )
else if ( ! latestGossip . members . exists ( _ . address == address ) ) {
2013-02-17 17:35:43 +01:00
// to support manual join when joining to seed nodes is stuck (no seed nodes available)
val snd = sender
seedNodeProcess match {
case Some ( `snd` ) ⇒
// seedNodeProcess completed, it will stop itself
seedNodeProcess = None
case Some ( s ) ⇒
// manual join, abort current seedNodeProcess
context stop s
seedNodeProcess = None
case None ⇒ // no seedNodeProcess in progress
}
2013-03-20 10:32:18 +01:00
// only wipe the state if we're not in the process of joining this address
if ( tryingToJoinWith . forall ( _ != address ) ) {
tryingToJoinWith = Some ( address )
// wipe our state since a node that joins a cluster must be empty
latestGossip = Gossip . empty
// wipe the failure detector since we are starting fresh and shouldn't care about the past
failureDetector . reset ( )
// wipe the publisher since we are starting fresh
publisher ! PublishStart
2012-07-05 13:55:08 +02:00
2013-03-20 10:32:18 +01:00
publish ( latestGossip )
}
2012-08-19 20:15:22 +02:00
context . become ( initialized )
if ( address == selfAddress )
2013-03-14 20:32:43 +01:00
joining ( address , cluster . selfRoles )
2012-08-19 20:15:22 +02:00
else
2013-03-14 20:32:43 +01:00
clusterCore ( address ) ! ClusterUserAction . Join ( selfAddress , cluster . selfRoles )
2012-08-19 20:15:22 +02:00
}
2012-07-05 13:55:08 +02:00
}
/* *
* State transition to JOINING - new node joining .
*/
2013-03-14 20:32:43 +01:00
def joining ( node : Address , roles : Set [ String ] ) : Unit = {
2013-02-17 17:35:43 +01:00
if ( node . protocol != selfAddress . protocol )
log . warning ( "Member with wrong protocol tried to join, but was ignored, expected [{}] but was [{}]" ,
selfAddress . protocol , node . protocol )
else if ( node . system != selfAddress . system )
log . warning ( "Member with wrong ActorSystem name tried to join, but was ignored, expected [{}] but was [{}]" ,
selfAddress . system , node . system )
else {
val localMembers = latestGossip . members
val localUnreachable = latestGossip . overview . unreachable
val alreadyMember = localMembers . exists ( _ . address == node )
2013-03-05 15:32:13 +01:00
val isUnreachable = localUnreachable . exists ( _ . address == node )
2013-02-17 17:35:43 +01:00
if ( ! alreadyMember && ! isUnreachable ) {
2013-03-05 15:32:13 +01:00
// remove the node from the failure detector
failureDetector . remove ( node )
2013-02-17 17:35:43 +01:00
// add joining node as Joining
// add self in case someone else joins before self has joined (Set discards duplicates)
2013-03-14 20:32:43 +01:00
val newMembers = localMembers + Member ( node , Joining , roles ) + Member ( selfAddress , Joining , cluster . selfRoles )
2013-03-05 15:32:13 +01:00
val newGossip = latestGossip copy ( members = newMembers )
2012-07-05 13:55:08 +02:00
2013-02-17 17:35:43 +01:00
val versionedGossip = newGossip : + vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
2012-07-05 13:55:08 +02:00
2013-02-17 17:35:43 +01:00
latestGossip = seenVersionedGossip
2012-07-05 13:55:08 +02:00
2013-03-14 20:32:43 +01:00
log . info ( "Cluster Node [{}] - Node [{}] is JOINING, roles [{}]" , selfAddress , node , roles . mkString ( ", " ) )
2013-02-17 17:35:43 +01:00
if ( node != selfAddress ) {
gossipTo ( node )
}
2012-07-05 13:55:08 +02:00
2013-02-17 17:35:43 +01:00
publish ( latestGossip )
2012-07-05 13:55:08 +02:00
}
}
}
/* *
* State transition to LEAVING .
*/
def leaving ( address : Address ) : Unit = {
2013-04-05 12:38:09 +02:00
// only try to update if the node is available (in the member ring)
if ( latestGossip . members . exists ( m ⇒ m . address == address && m . status == Up ) ) {
2013-03-14 20:32:43 +01:00
val newMembers = latestGossip . members map { m ⇒ if ( m . address == address ) m . copy ( status = Leaving ) else m } // mark node as LEAVING
2012-11-27 18:07:37 +01:00
val newGossip = latestGossip copy ( members = newMembers )
2012-07-05 13:55:08 +02:00
val versionedGossip = newGossip : + vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
latestGossip = seenVersionedGossip
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Marked address [{}] as [{}]" , selfAddress , address , Leaving )
2012-11-27 18:07:37 +01:00
publish ( latestGossip )
2012-07-05 13:55:08 +02:00
}
}
/* *
* State transition to EXITING .
*/
def exiting ( address : Address ) : Unit = {
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Marked node [{}] as [{}]" , selfAddress , address , Exiting )
2012-07-05 13:55:08 +02:00
// FIXME implement when we implement hand-off
}
/* *
* State transition to REMOVED .
*
* This method is for now only called after the LEADER have sent a Removed message - telling the node
* to shut down himself .
*
* In the future we might change this to allow the USER to send a Removed ( address ) message telling an
2013-04-09 21:06:48 +02:00
* arbitrary node to be moved directly from UP -> REMOVED .
2012-07-05 13:55:08 +02:00
*/
def removing ( address : Address ) : Unit = {
log . info ( "Cluster Node [{}] - Node has been REMOVED by the leader - shutting down..." , selfAddress )
2013-02-11 10:40:01 +01:00
cluster . shutdown ( )
2012-07-05 13:55:08 +02:00
}
/* *
* The node to DOWN is removed from the 'members' set and put in the 'unreachable' set ( if not already there )
* and its status is set to DOWN . The node is also removed from the 'seen' table .
*
* The node will reside as DOWN in the 'unreachable' set until an explicit command JOIN command is sent directly
* to this node and it will then go through the normal JOINING procedure .
*/
def downing ( address : Address ) : Unit = {
val localGossip = latestGossip
val localMembers = localGossip . members
val localOverview = localGossip . overview
val localSeen = localOverview . seen
val localUnreachableMembers = localOverview . unreachable
// 1. check if the node to DOWN is in the 'members' set
2012-07-07 20:55:02 +02:00
val downedMember : Option [ Member ] =
localMembers . collectFirst { case m if m . address == address ⇒ m . copy ( status = Down ) }
2012-07-05 13:55:08 +02:00
val newMembers = downedMember match {
case Some ( m ) ⇒
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Marking node [{}] as [{}]" , selfAddress , m . address , Down )
2012-07-05 13:55:08 +02:00
localMembers - m
case None ⇒ localMembers
}
// 2. check if the node to DOWN is in the 'unreachable' set
val newUnreachableMembers =
localUnreachableMembers . map { member ⇒
// no need to DOWN members already DOWN
if ( member . address == address && member . status != Down ) {
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Marking unreachable node [{}] as [{}]" , selfAddress , member . address , Down )
2012-07-05 13:55:08 +02:00
member copy ( status = Down )
} else member
}
// 3. add the newly DOWNED members from the 'members' (in step 1.) to the 'newUnreachableMembers' set.
val newUnreachablePlusNewlyDownedMembers = newUnreachableMembers ++ downedMember
// 4. remove nodes marked as DOWN from the 'seen' table
2012-07-07 20:55:02 +02:00
val newSeen = localSeen -- newUnreachablePlusNewlyDownedMembers . collect { case m if m . status == Down ⇒ m . address }
2012-07-05 13:55:08 +02:00
// update gossip overview
val newOverview = localOverview copy ( seen = newSeen , unreachable = newUnreachablePlusNewlyDownedMembers )
val newGossip = localGossip copy ( overview = newOverview , members = newMembers ) // update gossip
val versionedGossip = newGossip : + vclockNode
latestGossip = versionedGossip seen selfAddress
2012-11-27 18:07:37 +01:00
publish ( latestGossip )
2012-07-05 13:55:08 +02:00
}
/* *
* Receive new gossip .
*/
def receiveGossip ( envelope : GossipEnvelope ) : Unit = {
val from = envelope . from
val remoteGossip = envelope . gossip
val localGossip = latestGossip
if ( remoteGossip . overview . unreachable . exists ( _ . address == selfAddress ) ) {
2013-02-12 21:45:41 +01:00
log . debug ( "Ignoring received gossip with self [{}] as unreachable, from [{}]" , selfAddress , from )
} else if ( localGossip . overview . isNonDownUnreachable ( from ) ) {
log . debug ( "Ignoring received gossip from unreachable [{}] " , from )
} else {
2013-03-20 10:32:18 +01:00
// if we're in the remote gossip and not Removed, then we're not joining
if ( tryingToJoinWith . nonEmpty && remoteGossip . member ( selfAddress ) . status != Removed )
tryingToJoinWith = None
2013-03-05 12:49:35 +01:00
val comparison = remoteGossip . version tryCompareTo localGossip . version
2013-03-06 13:11:46 +01:00
val conflict = comparison . isEmpty
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
val ( winningGossip , talkback , newStats ) = comparison match {
case None ⇒
// conflicting versions, merge
( remoteGossip merge localGossip , true , stats . incrementMergeCount )
case Some ( 0 ) ⇒
// same version
( remoteGossip mergeSeen localGossip , ! remoteGossip . seenByAddress ( selfAddress ) , stats . incrementSameCount )
case Some ( x ) if x < 0 ⇒
// local is newer
( localGossip , true , stats . incrementNewerCount )
case _ ⇒
// remote is newer
( remoteGossip , ! remoteGossip . seenByAddress ( selfAddress ) , stats . incrementOlderCount )
}
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
stats = newStats
latestGossip = winningGossip seen selfAddress
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
// for all new joining nodes we remove them from the failure detector
2013-03-08 13:26:50 +01:00
latestGossip . members foreach {
node ⇒ if ( node . status == Joining && ! localGossip . members ( node ) ) failureDetector . remove ( node . address )
2013-03-07 13:20:50 +01:00
}
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
log . debug ( "Cluster Node [{}] - Receiving gossip from [{}]" , selfAddress , from )
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
if ( conflict ) {
log . debug (
"""Couldn't establish a causal relationship between "remote" gossip and "local" gossip - Remote[{}] - Local[{}] - merged them into [{}]""" ,
remoteGossip , localGossip , winningGossip )
}
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
stats = stats . incrementReceivedGossipCount
publish ( latestGossip )
2012-07-05 13:55:08 +02:00
2013-03-07 13:20:50 +01:00
if ( envelope . conversation && talkback ) {
// send back gossip to sender when sender had different view, i.e. merge, or sender had
// older or sender had newer
gossipTo ( from )
2012-07-05 13:55:08 +02:00
}
}
}
def mergeRate ( count : Long ) : Double = ( count * 1000.0 ) / GossipInterval . toMillis
/* *
* Initiates a new round of gossip .
*/
def gossip ( ) : Unit = {
log . debug ( "Cluster Node [{}] - Initiating new round of gossip" , selfAddress )
if ( ! isSingletonCluster && isAvailable ) {
val localGossip = latestGossip
2012-07-07 20:55:02 +02:00
val preferredGossipTargets =
if ( ThreadLocalRandom . current . nextDouble ( ) < GossipDifferentViewProbability ) { // If it's time to try to gossip to some nodes with a different view
// gossip to a random alive member with preference to a member with older or newer gossip version
val localMemberAddressesSet = localGossip . members map { _ . address }
val nodesWithDifferentView = for {
( address , version ) ← localGossip . overview . seen
if localMemberAddressesSet contains address
if version != localGossip . version
} yield address
nodesWithDifferentView . toIndexedSeq
} else Vector . empty [ Address ]
gossipToRandomNodeOf (
if ( preferredGossipTargets . nonEmpty ) preferredGossipTargets
2012-07-12 00:34:51 +02:00
else localGossip . members . toIndexedSeq . map ( _ . address ) // Fall back to localGossip; important to not accidentally use `map` of the SortedSet, since the original order is not preserved)
2012-07-07 20:55:02 +02:00
)
2012-07-05 13:55:08 +02:00
}
}
/* *
* Runs periodic leader actions , such as auto - downing unreachable nodes , assigning partitions etc .
*/
def leaderActions ( ) : Unit = {
val localGossip = latestGossip
val localMembers = localGossip . members
2012-08-19 21:48:39 +02:00
val isLeader = localGossip . isLeader ( selfAddress )
2012-07-05 13:55:08 +02:00
if ( isLeader && isAvailable ) {
// only run the leader actions if we are the LEADER and available
val localOverview = localGossip . overview
val localSeen = localOverview . seen
val localUnreachableMembers = localOverview . unreachable
val hasPartionHandoffCompletedSuccessfully : Boolean = {
// FIXME implement partion handoff and a check if it is completed - now just returns TRUE - e.g. has completed successfully
true
}
// Leader actions are as follows:
2012-07-07 20:55:02 +02:00
// 1. Move JOINING => UP -- When a node joins the cluster
// 2. Move LEAVING => EXITING -- When all partition handoff has completed
// 3. Non-exiting remain -- When all partition handoff has completed
// 4. Move EXITING => REMOVED -- When all nodes have seen that the node is EXITING (convergence) - remove the nodes from the node ring and seen table
// 5. Move UNREACHABLE => DOWN -- When the node is in the UNREACHABLE set it can be auto-down by leader
2013-03-05 15:32:13 +01:00
// 6. Move DOWN => REMOVED -- When all nodes have seen that the node is DOWN (convergence) - remove the nodes from the node ring and seen table
// 7. Updating the vclock version for the changes
// 8. Updating the 'seen' table
// 9. Try to update the state with the new gossip
// 10. If success - run all the side-effecting processing
2012-07-05 13:55:08 +02:00
val (
newGossip : Gossip ,
hasChangedState : Boolean ,
upMembers ,
exitingMembers ,
removedMembers ,
2013-04-09 21:06:48 +02:00
removedUnreachableMembers ,
2012-07-05 13:55:08 +02:00
unreachableButNotDownedMembers ) =
if ( localGossip . convergence ) {
// we have convergence - so we can't have unreachable nodes
2013-03-14 20:32:43 +01:00
def enoughMembers : Boolean = {
localMembers . size >= MinNrOfMembers && MinNrOfMembersOfRole . forall {
case ( role , threshold ) ⇒ localMembers . count ( _ . hasRole ( role ) ) >= threshold
}
}
def isJoiningToUp ( m : Member ) : Boolean = m . status == Joining && enoughMembers
2012-12-10 08:46:25 +01:00
2012-07-07 20:55:02 +02:00
// transform the node member ring
val newMembers = localMembers collect {
2013-03-05 15:32:13 +01:00
// Move JOINING => UP (once all nodes have seen that this node is JOINING, i.e. we have a convergence)
2012-12-10 08:46:25 +01:00
// and minimum number of nodes have joined the cluster
case member if isJoiningToUp ( member ) ⇒ member copy ( status = Up )
2013-03-05 15:32:13 +01:00
// Move LEAVING => EXITING (once we have a convergence on LEAVING
2012-12-10 08:46:25 +01:00
// *and* if we have a successful partition handoff)
case member if member . status == Leaving && hasPartionHandoffCompletedSuccessfully ⇒
member copy ( status = Exiting )
2013-03-05 15:32:13 +01:00
// Everyone else that is not Exiting stays as they are
case member if member . status != Exiting && member . status != Down ⇒ member
// Move EXITING => REMOVED, DOWN => REMOVED - i.e. remove the nodes from the 'members' set/node ring and seen table
2012-07-07 20:55:02 +02:00
}
2012-07-05 13:55:08 +02:00
// ----------------------
2013-03-05 15:32:13 +01:00
// Store away all stuff needed for the side-effecting processing
2012-07-05 13:55:08 +02:00
// ----------------------
// Check for the need to do side-effecting on successful state change
2013-03-05 15:32:13 +01:00
// Repeat the checking for transitions between JOINING -> UP, LEAVING -> EXITING, EXITING -> REMOVED, DOWN -> REMOVED
2012-07-05 13:55:08 +02:00
// to check for state-changes and to store away removed and exiting members for later notification
// 1. check for state-changes to update
2012-12-10 08:46:25 +01:00
// 2. store away removed and exiting members so we can separate the pure state changes
2013-03-05 15:32:13 +01:00
val ( removedMembers , newMembers1 ) = localMembers partition ( m ⇒ m . status == Exiting || m . status == Down )
2013-04-09 21:06:48 +02:00
val ( removedUnreachable , newUnreachable ) = localUnreachableMembers partition ( _ . status == Down )
2012-07-05 13:55:08 +02:00
2012-12-10 08:46:25 +01:00
val ( upMembers , newMembers2 ) = newMembers1 partition ( isJoiningToUp ( _ ) )
2012-07-05 13:55:08 +02:00
2012-07-07 20:55:02 +02:00
val exitingMembers = newMembers2 filter ( _ . status == Leaving && hasPartionHandoffCompletedSuccessfully )
2012-07-05 13:55:08 +02:00
2013-04-09 21:06:48 +02:00
val hasChangedState = removedMembers . nonEmpty || removedUnreachable . nonEmpty || upMembers . nonEmpty || exitingMembers . nonEmpty
2012-07-05 13:55:08 +02:00
// removing REMOVED nodes from the 'seen' table
2013-04-09 21:06:48 +02:00
val newSeen = localSeen -- removedMembers . map ( _ . address ) -- removedUnreachable . map ( _ . address )
2012-07-05 13:55:08 +02:00
2013-04-09 21:06:48 +02:00
val newOverview = localOverview copy ( seen = newSeen , unreachable = newUnreachable ) // update gossip overview
2012-07-05 13:55:08 +02:00
val newGossip = localGossip copy ( members = newMembers , overview = newOverview ) // update gossip
2013-04-09 21:06:48 +02:00
( newGossip , hasChangedState , upMembers , exitingMembers , removedMembers , removedUnreachable , Member . none )
2012-07-05 13:55:08 +02:00
} else if ( AutoDown ) {
// we don't have convergence - so we might have unreachable nodes
// if 'auto-down' is turned on, then try to auto-down any unreachable nodes
2012-07-07 20:55:02 +02:00
val newUnreachableMembers = localUnreachableMembers collect {
2012-07-05 13:55:08 +02:00
// ----------------------
2013-03-05 15:32:13 +01:00
// Move UNREACHABLE => DOWN (auto-downing by leader)
2012-07-05 13:55:08 +02:00
// ----------------------
2012-07-07 20:55:02 +02:00
case member if member . status != Down ⇒ member copy ( status = Down )
case downMember ⇒ downMember // no need to DOWN members already DOWN
2012-07-05 13:55:08 +02:00
}
// Check for the need to do side-effecting on successful state change
2012-07-07 20:55:02 +02:00
val unreachableButNotDownedMembers = localUnreachableMembers filter ( _ . status != Down )
2012-07-05 13:55:08 +02:00
// removing nodes marked as DOWN from the 'seen' table
val newSeen = localSeen -- newUnreachableMembers . collect { case m if m . status == Down ⇒ m . address }
val newOverview = localOverview copy ( seen = newSeen , unreachable = newUnreachableMembers ) // update gossip overview
val newGossip = localGossip copy ( overview = newOverview ) // update gossip
2013-04-09 21:06:48 +02:00
( newGossip , unreachableButNotDownedMembers . nonEmpty , Member . none , Member . none , Member . none , Member . none , unreachableButNotDownedMembers )
2012-07-05 13:55:08 +02:00
2013-04-09 21:06:48 +02:00
} else ( localGossip , false , Member . none , Member . none , Member . none , Member . none , Member . none )
2012-07-05 13:55:08 +02:00
if ( hasChangedState ) { // we have a change of state - version it and try to update
// ----------------------
2013-03-05 15:32:13 +01:00
// Updating the vclock version for the changes
2012-07-05 13:55:08 +02:00
// ----------------------
val versionedGossip = newGossip : + vclockNode
// ----------------------
2013-03-05 15:32:13 +01:00
// Updating the 'seen' table
// Unless the leader (this node) is part of the removed members, i.e. the leader have moved himself from EXITING -> REMOVED
2012-07-05 13:55:08 +02:00
// ----------------------
val seenVersionedGossip =
if ( removedMembers . exists ( _ . address == selfAddress ) ) versionedGossip
else versionedGossip seen selfAddress
// ----------------------
2013-03-05 15:32:13 +01:00
// Update the state with the new gossip
2012-07-05 13:55:08 +02:00
// ----------------------
latestGossip = seenVersionedGossip
// ----------------------
2013-03-05 15:32:13 +01:00
// Run all the side-effecting processing
2012-07-05 13:55:08 +02:00
// ----------------------
// log the move of members from joining to up
2013-03-08 09:39:48 +01:00
upMembers foreach { member ⇒
log . info ( "Cluster Node [{}] - Leader is moving node [{}] from [{}] to [{}]" ,
selfAddress , member . address , member . status , Up )
}
2012-07-05 13:55:08 +02:00
// tell all removed members to remove and shut down themselves
removedMembers foreach { member ⇒
val address = member . address
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Leader is moving node [{}] from [{}] to [{}] - and removing node from node ring" ,
selfAddress , address , member . status , Removed )
2013-01-25 15:03:52 +01:00
clusterCore ( address ) ! ClusterLeaderAction . Remove ( address )
2012-07-05 13:55:08 +02:00
}
// tell all exiting members to exit
exitingMembers foreach { member ⇒
val address = member . address
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Leader is moving node [{}] from [{}] to [{}]" ,
selfAddress , address , member . status , Exiting )
2013-01-25 15:03:52 +01:00
clusterCore ( address ) ! ClusterLeaderAction . Exit ( address ) // FIXME should use ? to await completion of handoff?
2012-07-05 13:55:08 +02:00
}
// log the auto-downing of the unreachable nodes
unreachableButNotDownedMembers foreach { member ⇒
2013-03-08 09:39:48 +01:00
log . info ( "Cluster Node [{}] - Leader is marking unreachable node [{}] as [{}]" , selfAddress , member . address , Down )
2012-07-05 13:55:08 +02:00
}
2013-04-09 21:06:48 +02:00
// log the auto-downing of the unreachable nodes
removedUnreachableMembers foreach { member ⇒
log . info ( "Cluster Node [{}] - Leader is removing unreachable node [{}]" , selfAddress , member . address )
}
2012-11-27 18:07:37 +01:00
publish ( latestGossip )
2012-07-05 13:55:08 +02:00
}
}
}
/* *
* Reaps the unreachable members ( moves them to the 'unreachable' list in the cluster overview ) according to the failure detector 's verdict .
*/
def reapUnreachableMembers ( ) : Unit = {
if ( ! isSingletonCluster && isAvailable ) {
// only scrutinize if we are a non-singleton cluster and available
val localGossip = latestGossip
val localOverview = localGossip . overview
val localMembers = localGossip . members
val localUnreachableMembers = localGossip . overview . unreachable
val newlyDetectedUnreachableMembers = localMembers filterNot { member ⇒
member . address == selfAddress || failureDetector . isAvailable ( member . address )
}
if ( newlyDetectedUnreachableMembers . nonEmpty ) {
val newMembers = localMembers -- newlyDetectedUnreachableMembers
val newUnreachableMembers = localUnreachableMembers ++ newlyDetectedUnreachableMembers
val newOverview = localOverview copy ( unreachable = newUnreachableMembers )
val newGossip = localGossip copy ( overview = newOverview , members = newMembers )
// updating vclock and 'seen' table
val versionedGossip = newGossip : + vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
latestGossip = seenVersionedGossip
log . error ( "Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]" , selfAddress , newlyDetectedUnreachableMembers . mkString ( ", " ) )
2012-11-27 18:07:37 +01:00
publish ( latestGossip )
2012-07-05 13:55:08 +02:00
}
}
}
def selectRandomNode ( addresses : IndexedSeq [ Address ] ) : Option [ Address ] =
if ( addresses . isEmpty ) None
else Some ( addresses ( ThreadLocalRandom . current nextInt addresses . size ) )
def isSingletonCluster : Boolean = latestGossip . isSingletonCluster
2012-12-06 15:26:57 +01:00
def isAvailable : Boolean = ! latestGossip . isUnreachable ( selfAddress )
2012-07-05 13:55:08 +02:00
/* *
* Gossips latest gossip to a random member in the set of members passed in as argument .
*
* @return the used [ [ akka . actor . Address ] if any
*/
2012-11-15 12:33:11 +01:00
private def gossipToRandomNodeOf ( addresses : immutable.IndexedSeq [ Address ] ) : Option [ Address ] = {
2012-07-05 13:55:08 +02:00
log . debug ( "Cluster Node [{}] - Selecting random node to gossip to [{}]" , selfAddress , addresses . mkString ( ", " ) )
// filter out myself
val peer = selectRandomNode ( addresses filterNot ( _ == selfAddress ) )
peer foreach gossipTo
peer
}
/* *
* Gossips latest gossip to an address .
*/
def gossipTo ( address : Address ) : Unit =
gossipTo ( address , GossipEnvelope ( selfAddress , latestGossip , conversation = true ) )
def oneWayGossipTo ( address : Address ) : Unit =
gossipTo ( address , GossipEnvelope ( selfAddress , latestGossip , conversation = false ) )
2013-01-17 14:00:01 +01:00
def gossipTo ( address : Address , gossipMsg : GossipEnvelope ) : Unit =
2013-02-12 21:45:41 +01:00
if ( address != selfAddress && gossipMsg . gossip . members . exists ( _ . address == address ) )
clusterCore ( address ) ! gossipMsg
2012-07-05 13:55:08 +02:00
2012-11-27 18:07:37 +01:00
def publish ( newGossip : Gossip ) : Unit = {
publisher ! PublishChanges ( newGossip )
2012-08-15 16:47:34 +02:00
if ( PublishStatsInterval == Duration . Zero ) publishInternalStats ( )
2012-08-14 10:58:30 +02:00
}
2012-08-19 20:15:22 +02:00
def publishInternalStats ( ) : Unit = publisher ! CurrentInternalStats ( stats )
2012-08-14 15:33:34 +02:00
2012-07-05 13:55:08 +02:00
}
2012-08-14 17:26:33 +02:00
/* *
* INTERNAL API .
2012-08-15 08:21:34 +02:00
*
2013-02-17 17:35:43 +01:00
* Used only for the first seed node .
* Sends InitJoin to all seed nodes ( except itself ) .
* If other seed nodes are not part of the cluster yet they will reply with
* InitJoinNack or not respond at all and then the first seed node
* will join itself to initialize the new cluster . When the first
* seed node is restarted , and some other seed node is part of the cluster
* it will reply with InitJoinAck and then the first seed node will join
* that other seed node to join existing cluster .
*/
private [ cluster ] final class FirstSeedNodeProcess ( seedNodes : immutable.IndexedSeq [ Address ] ) extends Actor with ActorLogging {
import InternalClusterAction._
val cluster = Cluster ( context . system )
def selfAddress = cluster . selfAddress
if ( seedNodes . size <= 1 || seedNodes . head != selfAddress )
throw new IllegalArgumentException ( "Join seed node should not be done" )
val timeout = Deadline . now + cluster . settings . SeedNodeTimeout
var remainingSeedNodes = seedNodes . toSet - selfAddress
// retry until one ack, or all nack, or timeout
import context.dispatcher
val retryTask = cluster . scheduler . schedule ( 1. second , 1. second , self , JoinSeedNode )
self ! JoinSeedNode
override def postStop ( ) : Unit = retryTask . cancel ( )
def receive = {
case JoinSeedNode ⇒
if ( timeout . hasTimeLeft ) {
// send InitJoin to remaining seed nodes (except myself)
2013-03-26 18:17:50 +01:00
remainingSeedNodes foreach { a ⇒ context . actorSelection ( context . parent . path . toStringWithAddress ( a ) ) ! InitJoin }
2013-02-17 17:35:43 +01:00
} else {
// no InitJoinAck received, initialize new cluster by joining myself
context . parent ! JoinTo ( selfAddress )
context . stop ( self )
}
case InitJoinAck ( address ) ⇒
// first InitJoinAck reply, join existing cluster
context . parent ! JoinTo ( address )
context . stop ( self )
case InitJoinNack ( address ) ⇒
remainingSeedNodes -= address
if ( remainingSeedNodes . isEmpty ) {
// initialize new cluster by joining myself when nacks from all other seed nodes
context . parent ! JoinTo ( selfAddress )
context . stop ( self )
}
}
}
/* *
* INTERNAL API .
*
* Sends InitJoin to all seed nodes ( except itself ) and expect
2012-08-15 08:21:34 +02:00
* InitJoinAck reply back . The seed node that replied first
* will be used , joined to . InitJoinAck replies received after the
* first one are ignored .
*
* Retries if no InitJoinAck replies are received within the
* SeedNodeTimeout .
* When at least one reply has been received it stops itself after
* an idle SeedNodeTimeout .
*
2012-08-31 12:27:17 +02:00
* The seed nodes can be started in any order , but they will not be "active" ,
* until they have been able to join another seed node ( seed1 ) .
* They will retry the join procedure .
* So one possible startup scenario is :
* 1. seed2 started , but doesn 't get any ack from seed1 or seed3
* 2. seed3 started , doesn 't get any ack from seed1 or seed3 ( seed2 doesn 't reply )
* 3. seed1 is started and joins itself
* 4. seed2 retries the join procedure and gets an ack from seed1 , and then joins to seed1
* 5. seed3 retries the join procedure and gets acks from seed2 first , and then joins to seed2
*
2012-08-14 17:26:33 +02:00
*/
2012-11-15 12:33:11 +01:00
private [ cluster ] final class JoinSeedNodeProcess ( seedNodes : immutable.IndexedSeq [ Address ] ) extends Actor with ActorLogging {
2012-08-14 17:26:33 +02:00
import InternalClusterAction._
2012-09-06 21:48:40 +02:00
def selfAddress = Cluster ( context . system ) . selfAddress
2012-08-14 17:26:33 +02:00
2012-09-06 21:48:40 +02:00
if ( seedNodes . isEmpty || seedNodes . head == selfAddress )
2012-08-14 17:26:33 +02:00
throw new IllegalArgumentException ( "Join seed node should not be done" )
2012-09-06 21:48:40 +02:00
context . setReceiveTimeout ( Cluster ( context . system ) . settings . SeedNodeTimeout )
2012-08-15 08:21:34 +02:00
2012-08-15 17:31:36 +02:00
override def preStart ( ) : Unit = self ! JoinSeedNode
2012-08-14 17:26:33 +02:00
def receive = {
case JoinSeedNode ⇒
2012-08-15 08:21:34 +02:00
// send InitJoin to all seed nodes (except myself)
2012-09-06 21:48:40 +02:00
seedNodes . collect {
2013-03-26 18:17:50 +01:00
case a if a != selfAddress ⇒ context . actorSelection ( context . parent . path . toStringWithAddress ( a ) )
2012-08-15 17:31:36 +02:00
} foreach { _ ! InitJoin }
2012-08-14 17:26:33 +02:00
case InitJoinAck ( address ) ⇒
2012-08-15 08:21:34 +02:00
// first InitJoinAck reply
2012-08-14 17:26:33 +02:00
context . parent ! JoinTo ( address )
2012-08-15 08:21:34 +02:00
context . become ( done )
2013-02-17 17:35:43 +01:00
case InitJoinNack ( _ ) ⇒ // that seed was uninitialized
2012-08-15 08:21:34 +02:00
case ReceiveTimeout ⇒
// no InitJoinAck received, try again
self ! JoinSeedNode
2012-08-14 17:26:33 +02:00
}
2012-08-15 08:21:34 +02:00
def done : Actor . Receive = {
case InitJoinAck ( _ ) ⇒ // already received one, skip rest
case ReceiveTimeout ⇒ context . stop ( self )
}
2012-08-14 17:26:33 +02:00
}
2012-12-10 08:46:25 +01:00
/* *
* INTERNAL API
*
* The supplied callback will be run , once , when current cluster member is `Up` .
*/
private [ cluster ] class OnMemberUpListener ( callback : Runnable ) extends Actor with ActorLogging {
import ClusterEvent._
val cluster = Cluster ( context . system )
// subscribe to MemberUp, re-subscribe when restart
override def preStart ( ) : Unit =
cluster . subscribe ( self , classOf [ MemberUp ] )
override def postStop ( ) : Unit =
cluster . unsubscribe ( self )
def receive = {
case state : CurrentClusterState ⇒
if ( state . members . exists ( isSelfUp ( _ ) ) )
done ( )
case MemberUp ( m ) ⇒
if ( isSelfUp ( m ) )
done ( )
}
def done ( ) : Unit = {
try callback . run ( ) catch {
case NonFatal ( e ) ⇒ log . error ( e , "OnMemberUp callback failed with [{}]" , e . getMessage )
} finally {
context stop self
}
}
def isSelfUp ( m : Member ) : Boolean =
m . address == cluster . selfAddress && m . status == MemberStatus . Up
}
2012-07-05 13:55:08 +02:00
/* *
* INTERNAL API
*/
2013-04-04 17:56:29 +02:00
@SerialVersionUID ( 1L )
2012-07-05 13:55:08 +02:00
private [ cluster ] case class ClusterStats (
receivedGossipCount : Long = 0L ,
mergeCount : Long = 0L ,
2013-03-05 12:49:35 +01:00
sameCount : Long = 0L ,
newerCount : Long = 0L ,
olderCount : Long = 0L ) {
2012-07-05 13:55:08 +02:00
def incrementReceivedGossipCount ( ) : ClusterStats =
copy ( receivedGossipCount = receivedGossipCount + 1 )
def incrementMergeCount ( ) : ClusterStats =
copy ( mergeCount = mergeCount + 1 )
2013-03-05 12:49:35 +01:00
def incrementSameCount ( ) : ClusterStats =
copy ( sameCount = sameCount + 1 )
def incrementNewerCount ( ) : ClusterStats =
copy ( newerCount = newerCount + 1 )
def incrementOlderCount ( ) : ClusterStats =
copy ( olderCount = olderCount + 1 )
2013-03-06 13:11:46 +01:00
def : + ( that : ClusterStats ) : ClusterStats = {
2013-03-05 12:49:35 +01:00
ClusterStats (
this . receivedGossipCount + that . receivedGossipCount ,
this . mergeCount + that . mergeCount ,
this . sameCount + that . sameCount ,
this . newerCount + that . newerCount ,
this . olderCount + that . olderCount )
}
2013-03-06 13:11:46 +01:00
def : - ( that : ClusterStats ) : ClusterStats = {
2013-03-05 12:49:35 +01:00
ClusterStats (
this . receivedGossipCount - that . receivedGossipCount ,
this . mergeCount - that . mergeCount ,
this . sameCount - that . sameCount ,
this . newerCount - that . newerCount ,
this . olderCount - that . olderCount )
}
2012-09-21 14:50:06 +02:00
}