2013-01-14 14:09:53 +01:00
|
|
|
/**
|
2015-03-07 22:58:48 -08:00
|
|
|
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
2013-01-14 14:09:53 +01:00
|
|
|
*/
|
|
|
|
|
|
2015-04-27 14:25:10 +02:00
|
|
|
package akka.cluster.singleton
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2015-04-29 18:23:45 +02:00
|
|
|
import com.typesafe.config.Config
|
2013-01-14 14:09:53 +01:00
|
|
|
import scala.concurrent.duration._
|
2013-04-28 22:05:40 +02:00
|
|
|
import scala.collection.immutable
|
2013-01-14 14:09:53 +01:00
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.Actor.Receive
|
2013-08-23 14:39:21 +02:00
|
|
|
import akka.actor.Deploy
|
2013-01-14 14:09:53 +01:00
|
|
|
import akka.actor.ActorLogging
|
2015-04-29 18:23:45 +02:00
|
|
|
import akka.actor.ActorSystem
|
2013-01-14 14:09:53 +01:00
|
|
|
import akka.actor.ActorRef
|
2013-03-26 18:17:50 +01:00
|
|
|
import akka.actor.ActorSelection
|
2013-01-14 14:09:53 +01:00
|
|
|
import akka.actor.Address
|
|
|
|
|
import akka.actor.FSM
|
|
|
|
|
import akka.actor.Props
|
|
|
|
|
import akka.actor.Terminated
|
|
|
|
|
import akka.cluster.Cluster
|
|
|
|
|
import akka.cluster.ClusterEvent._
|
2013-04-28 22:05:40 +02:00
|
|
|
import akka.cluster.Member
|
|
|
|
|
import akka.cluster.MemberStatus
|
2013-01-14 14:09:53 +01:00
|
|
|
import akka.AkkaException
|
2015-04-29 18:23:45 +02:00
|
|
|
import akka.actor.NoSerializationVerificationNeeded
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2015-04-29 18:23:45 +02:00
|
|
|
object ClusterSingletonManagerSettings {
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-17 22:14:19 +02:00
|
|
|
/**
|
2015-04-29 18:23:45 +02:00
|
|
|
* Create settings from the default configuration
|
|
|
|
|
* `akka.cluster.singleton`.
|
2013-04-17 22:14:19 +02:00
|
|
|
*/
|
2015-04-29 18:23:45 +02:00
|
|
|
def apply(system: ActorSystem): ClusterSingletonManagerSettings =
|
|
|
|
|
apply(system.settings.config.getConfig("akka.cluster.singleton"))
|
2013-04-17 22:14:19 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-04-29 18:23:45 +02:00
|
|
|
* Create settings from a configuration with the same layout as
|
|
|
|
|
* the default configuration `akka.cluster.singleton`.
|
2013-04-17 22:14:19 +02:00
|
|
|
*/
|
2015-04-29 18:23:45 +02:00
|
|
|
def apply(config: Config): ClusterSingletonManagerSettings =
|
|
|
|
|
new ClusterSingletonManagerSettings(
|
|
|
|
|
singletonName = config.getString("singleton-name"),
|
|
|
|
|
role = roleOption(config.getString("role")),
|
|
|
|
|
maxHandOverRetries = config.getInt("max-hand-over-retries"),
|
|
|
|
|
maxTakeOverRetries = config.getInt("max-take-over-retries"),
|
|
|
|
|
retryInterval = config.getDuration("retry-interval", MILLISECONDS).millis)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Java API: Create settings from the default configuration
|
|
|
|
|
* `akka.cluster.singleton`.
|
|
|
|
|
*/
|
|
|
|
|
def create(system: ActorSystem): ClusterSingletonManagerSettings = apply(system)
|
2013-04-17 22:14:19 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-04-29 18:23:45 +02:00
|
|
|
* Java API: Create settings from a configuration with the same layout as
|
|
|
|
|
* the default configuration `akka.cluster.singleton`.
|
|
|
|
|
*/
|
|
|
|
|
def create(config: Config): ClusterSingletonManagerSettings = apply(config)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
2013-04-17 22:14:19 +02:00
|
|
|
*/
|
2015-04-29 18:23:45 +02:00
|
|
|
private[akka] def roleOption(role: String): Option[String] =
|
|
|
|
|
if (role == "") None else Option(role)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* @param singletonName The actor name of the child singleton actor.
|
|
|
|
|
*
|
|
|
|
|
* @param role Singleton among the nodes tagged with specified role.
|
|
|
|
|
* If the role is not specified it's a singleton among all nodes in
|
|
|
|
|
* the cluster.
|
|
|
|
|
*
|
|
|
|
|
* @param maxHandOverRetries When a node is becoming oldest it sends
|
|
|
|
|
* hand-over request to previous oldest. This is retried with the
|
|
|
|
|
* `retryInterval` until the previous oldest confirms that the hand
|
|
|
|
|
* over has started, or this `maxHandOverRetries` limit has been
|
|
|
|
|
* reached. If the retry limit is reached it takes the decision to be
|
|
|
|
|
* the new oldest if previous oldest is unknown (typically removed),
|
|
|
|
|
* otherwise it initiates a new round by throwing
|
|
|
|
|
* [[akka.cluster.singleton.ClusterSingletonManagerIsStuck]] and expecting
|
|
|
|
|
* restart with fresh state. For a cluster with many members you might
|
|
|
|
|
* need to increase this retry limit because it takes longer time to
|
|
|
|
|
* propagate changes across all nodes.
|
|
|
|
|
*
|
|
|
|
|
* @param maxTakeOverRetries When a oldest node leaves the cluster it is
|
|
|
|
|
* not oldest any more and then it sends take over request to the new oldest to
|
|
|
|
|
* initiate the hand-over process. This is retried with the `retryInterval` until
|
|
|
|
|
* this retry limit has been reached. If the retry limit is reached it initiates
|
|
|
|
|
* a new round by throwing [[akka.cluster.singleton.ClusterSingletonManagerIsStuck]]
|
|
|
|
|
* and expecting restart with fresh state. This will also cause the singleton actor
|
|
|
|
|
* to be stopped. `maxTakeOverRetries` must be less than `maxHandOverRetries` to
|
|
|
|
|
* ensure that new oldest doesn't start singleton actor before previous is
|
|
|
|
|
* stopped for certain corner cases.
|
|
|
|
|
*
|
|
|
|
|
* @param retryInterval Interval for hand over and take over messages
|
|
|
|
|
*/
|
|
|
|
|
final class ClusterSingletonManagerSettings(
|
|
|
|
|
val singletonName: String,
|
|
|
|
|
val role: Option[String],
|
|
|
|
|
val maxHandOverRetries: Int,
|
|
|
|
|
val maxTakeOverRetries: Int,
|
|
|
|
|
val retryInterval: FiniteDuration) extends NoSerializationVerificationNeeded {
|
|
|
|
|
|
|
|
|
|
// to ensure that new oldest doesn't start singleton actor before previous is stopped for certain corner cases
|
|
|
|
|
require(maxTakeOverRetries < maxHandOverRetries,
|
|
|
|
|
s"maxTakeOverRetries [${maxTakeOverRetries}]must be < maxHandOverRetries [${maxHandOverRetries}]")
|
|
|
|
|
|
|
|
|
|
def withSingletonName(name: String): ClusterSingletonManagerSettings = copy(singletonName = name)
|
|
|
|
|
|
|
|
|
|
def withRole(role: String): ClusterSingletonManagerSettings = copy(role = ClusterSingletonManagerSettings.roleOption(role))
|
|
|
|
|
|
|
|
|
|
def withRole(role: Option[String]) = copy(role = role)
|
|
|
|
|
|
|
|
|
|
def withRetry(maxHandOverRetries: Int, maxTakeOverRetries: Int, retryInterval: FiniteDuration): ClusterSingletonManagerSettings =
|
|
|
|
|
copy(maxHandOverRetries = maxHandOverRetries,
|
|
|
|
|
maxTakeOverRetries = maxTakeOverRetries,
|
|
|
|
|
retryInterval = retryInterval)
|
|
|
|
|
|
|
|
|
|
private def copy(singletonName: String = singletonName,
|
|
|
|
|
role: Option[String] = role,
|
|
|
|
|
maxHandOverRetries: Int = maxHandOverRetries,
|
|
|
|
|
maxTakeOverRetries: Int = maxTakeOverRetries,
|
|
|
|
|
retryInterval: FiniteDuration = retryInterval): ClusterSingletonManagerSettings =
|
|
|
|
|
new ClusterSingletonManagerSettings(singletonName, role, maxHandOverRetries, maxTakeOverRetries, retryInterval)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object ClusterSingletonManager {
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Scala API: Factory method for `ClusterSingletonManager` [[akka.actor.Props]].
|
|
|
|
|
*/
|
|
|
|
|
def props(
|
2013-09-10 13:35:51 +02:00
|
|
|
singletonProps: Props,
|
2013-04-17 22:14:19 +02:00
|
|
|
terminationMessage: Any,
|
2015-04-29 18:23:45 +02:00
|
|
|
settings: ClusterSingletonManagerSettings): Props =
|
|
|
|
|
Props(new ClusterSingletonManager(singletonProps, terminationMessage, settings)).withDeploy(Deploy.local)
|
2013-04-17 22:14:19 +02:00
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
/**
|
2013-02-08 13:13:52 +01:00
|
|
|
* INTERNAL API
|
2013-01-14 14:09:53 +01:00
|
|
|
* public due to the `with FSM` type parameters
|
|
|
|
|
*/
|
|
|
|
|
sealed trait State
|
|
|
|
|
/**
|
2013-02-08 13:13:52 +01:00
|
|
|
* INTERNAL API
|
2013-01-14 14:09:53 +01:00
|
|
|
* public due to the `with FSM` type parameters
|
|
|
|
|
*/
|
|
|
|
|
sealed trait Data
|
|
|
|
|
|
|
|
|
|
/**
|
2013-02-08 13:13:52 +01:00
|
|
|
* INTERNAL API
|
2013-01-14 14:09:53 +01:00
|
|
|
*/
|
|
|
|
|
private object Internal {
|
|
|
|
|
/**
|
2013-04-28 22:05:40 +02:00
|
|
|
* Sent from new oldest to previous oldest to initate the
|
2013-01-28 08:47:52 +01:00
|
|
|
* hand-over process. `HandOverInProgress` and `HandOverDone`
|
2013-01-14 14:09:53 +01:00
|
|
|
* are expected replies.
|
|
|
|
|
*/
|
|
|
|
|
case object HandOverToMe
|
|
|
|
|
/**
|
2013-04-28 22:05:40 +02:00
|
|
|
* Confirmation by the previous oldest that the hand
|
2013-01-28 08:47:52 +01:00
|
|
|
* over process, shut down of the singleton actor, has
|
2013-01-14 14:09:53 +01:00
|
|
|
* started.
|
|
|
|
|
*/
|
|
|
|
|
case object HandOverInProgress
|
|
|
|
|
/**
|
2013-04-28 22:05:40 +02:00
|
|
|
* Confirmation by the previous oldest that the singleton
|
2013-01-28 08:47:52 +01:00
|
|
|
* actor has been terminated and the hand-over process is
|
2013-09-10 13:35:51 +02:00
|
|
|
* completed.
|
2013-01-14 14:09:53 +01:00
|
|
|
*/
|
2013-09-10 13:35:51 +02:00
|
|
|
case object HandOverDone
|
2013-01-14 14:09:53 +01:00
|
|
|
/**
|
2013-04-28 22:05:40 +02:00
|
|
|
* Sent from from previous oldest to new oldest to
|
2013-01-28 08:47:52 +01:00
|
|
|
* initiate the normal hand-over process.
|
2013-01-14 14:09:53 +01:00
|
|
|
* Especially useful when new node joins and becomes
|
2013-04-28 22:05:40 +02:00
|
|
|
* oldest immediately, without knowing who was previous
|
|
|
|
|
* oldest.
|
2013-01-14 14:09:53 +01:00
|
|
|
*/
|
|
|
|
|
case object TakeOverFromMe
|
|
|
|
|
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class HandOverRetry(count: Int)
|
|
|
|
|
final case class TakeOverRetry(count: Int)
|
2013-01-14 14:09:53 +01:00
|
|
|
case object Cleanup
|
2013-04-28 22:05:40 +02:00
|
|
|
case object StartOldestChangedBuffer
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
case object Start extends State
|
2013-04-28 22:05:40 +02:00
|
|
|
case object Oldest extends State
|
|
|
|
|
case object Younger extends State
|
|
|
|
|
case object BecomingOldest extends State
|
|
|
|
|
case object WasOldest extends State
|
2013-01-14 14:09:53 +01:00
|
|
|
case object HandingOver extends State
|
|
|
|
|
case object TakeOver extends State
|
2013-05-09 09:49:59 +02:00
|
|
|
case object End extends State
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
case object Uninitialized extends Data
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class YoungerData(oldestOption: Option[Address]) extends Data
|
|
|
|
|
final case class BecomingOldestData(previousOldestOption: Option[Address]) extends Data
|
|
|
|
|
final case class OldestData(singleton: ActorRef, singletonTerminated: Boolean = false) extends Data
|
|
|
|
|
final case class WasOldestData(singleton: ActorRef, singletonTerminated: Boolean,
|
|
|
|
|
newOldestOption: Option[Address]) extends Data
|
|
|
|
|
final case class HandingOverData(singleton: ActorRef, handOverTo: Option[ActorRef]) extends Data
|
2013-05-09 09:49:59 +02:00
|
|
|
case object EndData extends Data
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
val HandOverRetryTimer = "hand-over-retry"
|
|
|
|
|
val TakeOverRetryTimer = "take-over-retry"
|
|
|
|
|
val CleanupTimer = "cleanup"
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
object OldestChangedBuffer {
|
2013-01-14 14:09:53 +01:00
|
|
|
/**
|
|
|
|
|
* Request to deliver one more event.
|
|
|
|
|
*/
|
|
|
|
|
case object GetNext
|
|
|
|
|
/**
|
|
|
|
|
* The first event, corresponding to CurrentClusterState.
|
|
|
|
|
*/
|
2015-02-09 15:34:58 +01:00
|
|
|
final case class InitialOldestState(oldest: Option[Address], safeToBeOldest: Boolean)
|
2013-04-28 22:05:40 +02:00
|
|
|
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class OldestChanged(oldest: Option[Address])
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2013-04-28 22:05:40 +02:00
|
|
|
* Notifications of member events that track oldest member is tunneled
|
2013-01-14 14:09:53 +01:00
|
|
|
* via this actor (child of ClusterSingletonManager) to be able to deliver
|
2013-04-28 22:05:40 +02:00
|
|
|
* one change at a time. Avoiding simultaneous changes simplifies
|
2013-01-14 14:09:53 +01:00
|
|
|
* the process in ClusterSingletonManager. ClusterSingletonManager requests
|
|
|
|
|
* next event with `GetNext` when it is ready for it. Only one outstanding
|
|
|
|
|
* `GetNext` request is allowed. Incoming events are buffered and delivered
|
|
|
|
|
* upon `GetNext` request.
|
|
|
|
|
*/
|
2013-04-28 22:05:40 +02:00
|
|
|
class OldestChangedBuffer(role: Option[String]) extends Actor {
|
|
|
|
|
import OldestChangedBuffer._
|
2013-01-14 14:09:53 +01:00
|
|
|
import context.dispatcher
|
|
|
|
|
|
|
|
|
|
val cluster = Cluster(context.system)
|
2013-04-28 22:05:40 +02:00
|
|
|
// sort by age, oldest first
|
|
|
|
|
val ageOrdering = Ordering.fromLessThan[Member] { (a, b) ⇒ a.isOlderThan(b) }
|
|
|
|
|
var membersByAge: immutable.SortedSet[Member] = immutable.SortedSet.empty(ageOrdering)
|
|
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
var changes = Vector.empty[AnyRef]
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
// subscribe to MemberEvent, re-subscribe when restart
|
|
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
cluster.subscribe(self, classOf[MemberEvent])
|
2013-03-14 20:32:43 +01:00
|
|
|
}
|
2013-01-14 14:09:53 +01:00
|
|
|
override def postStop(): Unit = cluster.unsubscribe(self)
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
def matchingRole(member: Member): Boolean = role match {
|
|
|
|
|
case None ⇒ true
|
|
|
|
|
case Some(r) ⇒ member.hasRole(r)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def trackChange(block: () ⇒ Unit): Unit = {
|
|
|
|
|
val before = membersByAge.headOption
|
|
|
|
|
block()
|
|
|
|
|
val after = membersByAge.headOption
|
|
|
|
|
if (before != after)
|
|
|
|
|
changes :+= OldestChanged(after.map(_.address))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def handleInitial(state: CurrentClusterState): Unit = {
|
2013-11-19 15:53:40 +01:00
|
|
|
membersByAge = immutable.SortedSet.empty(ageOrdering) ++ state.members.filter(m ⇒
|
2015-02-09 15:34:58 +01:00
|
|
|
(m.status == MemberStatus.Up || m.status == MemberStatus.Leaving) && matchingRole(m))
|
|
|
|
|
val safeToBeOldest = !state.members.exists { m ⇒ (m.status == MemberStatus.Down || m.status == MemberStatus.Exiting) }
|
|
|
|
|
val initial = InitialOldestState(membersByAge.headOption.map(_.address), safeToBeOldest)
|
2013-04-28 22:05:40 +02:00
|
|
|
changes :+= initial
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def add(m: Member): Unit = {
|
|
|
|
|
if (matchingRole(m))
|
|
|
|
|
trackChange { () ⇒ membersByAge += m }
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def remove(m: Member): Unit = {
|
|
|
|
|
if (matchingRole(m))
|
|
|
|
|
trackChange { () ⇒ membersByAge -= m }
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def sendFirstChange(): Unit = {
|
|
|
|
|
val event = changes.head
|
|
|
|
|
changes = changes.tail
|
|
|
|
|
context.parent ! event
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
def receive = {
|
2013-04-28 22:05:40 +02:00
|
|
|
case state: CurrentClusterState ⇒ handleInitial(state)
|
|
|
|
|
case MemberUp(m) ⇒ add(m)
|
|
|
|
|
case mEvent: MemberEvent if (mEvent.isInstanceOf[MemberExited] || mEvent.isInstanceOf[MemberRemoved]) ⇒
|
|
|
|
|
remove(mEvent.member)
|
2013-01-14 14:09:53 +01:00
|
|
|
case GetNext if changes.isEmpty ⇒
|
|
|
|
|
context.become(deliverNext, discardOld = false)
|
|
|
|
|
case GetNext ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
sendFirstChange()
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// the buffer was empty when GetNext was received, deliver next event immediately
|
|
|
|
|
def deliverNext: Actor.Receive = {
|
|
|
|
|
case state: CurrentClusterState ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
handleInitial(state)
|
|
|
|
|
sendFirstChange()
|
2013-01-14 14:09:53 +01:00
|
|
|
context.unbecome()
|
2013-04-28 22:05:40 +02:00
|
|
|
case MemberUp(m) ⇒
|
|
|
|
|
add(m)
|
|
|
|
|
if (changes.nonEmpty) {
|
|
|
|
|
sendFirstChange()
|
|
|
|
|
context.unbecome()
|
|
|
|
|
}
|
|
|
|
|
case mEvent: MemberEvent if (mEvent.isInstanceOf[MemberExited] || mEvent.isInstanceOf[MemberRemoved]) ⇒
|
|
|
|
|
remove(mEvent.member)
|
|
|
|
|
if (changes.nonEmpty) {
|
|
|
|
|
sendFirstChange()
|
2013-03-14 20:32:43 +01:00
|
|
|
context.unbecome()
|
|
|
|
|
}
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Thrown when a consistent state can't be determined within the
|
|
|
|
|
* defined retry limits. Eventually it will reach a stable state and
|
|
|
|
|
* can continue, and that is simplified by starting over with a clean
|
|
|
|
|
* state. Parent supervisor should typically restart the actor, i.e.
|
|
|
|
|
* default decision.
|
|
|
|
|
*/
|
|
|
|
|
class ClusterSingletonManagerIsStuck(message: String) extends AkkaException(message, null)
|
|
|
|
|
|
|
|
|
|
/**
|
2013-03-14 20:32:43 +01:00
|
|
|
* Manages singleton actor instance among all cluster nodes or a group
|
|
|
|
|
* of nodes tagged with a specific role. At most one singleton instance
|
|
|
|
|
* is running at any point in time.
|
|
|
|
|
*
|
|
|
|
|
* The ClusterSingletonManager is supposed to be started on all nodes,
|
|
|
|
|
* or all nodes with specified role, in the cluster with `actorOf`.
|
2013-04-28 22:05:40 +02:00
|
|
|
* The actual singleton is started on the oldest node by creating a child
|
2013-01-14 14:09:53 +01:00
|
|
|
* actor from the supplied `singletonProps`.
|
|
|
|
|
*
|
2015-04-13 10:07:14 +02:00
|
|
|
* The singleton actor is always running on the oldest member with specified role.
|
|
|
|
|
* The oldest member is determined by [[akka.cluster.Member#isOlderThan]].
|
2013-04-28 22:05:40 +02:00
|
|
|
* This can change when removing members. A graceful hand over can normally
|
|
|
|
|
* be performed when current oldest node is leaving the cluster. Be aware that
|
|
|
|
|
* there is a short time period when there is no active singleton during the
|
2013-01-28 08:47:52 +01:00
|
|
|
* hand-over process.
|
2013-01-14 14:09:53 +01:00
|
|
|
*
|
2013-04-28 22:05:40 +02:00
|
|
|
* The cluster failure detector will notice when oldest node
|
2013-01-28 08:47:52 +01:00
|
|
|
* becomes unreachable due to things like JVM crash, hard shut down,
|
2013-11-19 15:53:40 +01:00
|
|
|
* or network failure. When the crashed node has been removed (via down) from the
|
|
|
|
|
* cluster then a new oldest node will take over and a new singleton actor is
|
|
|
|
|
* created. For these failure scenarios there will not be a graceful hand-over,
|
|
|
|
|
* but more than one active singletons is prevented by all reasonable means. Some
|
|
|
|
|
* corner cases are eventually resolved by configurable timeouts.
|
2013-01-14 14:09:53 +01:00
|
|
|
*
|
2015-06-04 21:21:37 +02:00
|
|
|
* You access the singleton actor with [[ClusterSingletonProxy]].
|
2013-04-28 22:05:40 +02:00
|
|
|
* Alternatively the singleton actor may broadcast its existence when it is started.
|
2013-01-14 14:09:53 +01:00
|
|
|
*
|
2014-05-07 11:09:53 +05:30
|
|
|
* Use factory method [[ClusterSingletonManager#props]] to create the
|
2013-04-17 22:14:19 +02:00
|
|
|
* [[akka.actor.Props]] for the actor.
|
|
|
|
|
*
|
2013-01-14 14:09:53 +01:00
|
|
|
*
|
2015-04-29 18:23:45 +02:00
|
|
|
* @param singletonProps [[akka.actor.Props]] of the singleton actor instance.
|
2013-01-14 14:09:53 +01:00
|
|
|
*
|
2015-04-29 18:23:45 +02:00
|
|
|
* @param terminationMessage When handing over to a new oldest node
|
2013-01-14 14:09:53 +01:00
|
|
|
* this `terminationMessage` is sent to the singleton actor to tell
|
2013-09-10 13:35:51 +02:00
|
|
|
* it to finish its work, close resources, and stop.
|
2013-04-28 22:05:40 +02:00
|
|
|
* The hand-over to the new oldest node is completed when the
|
2013-01-14 14:09:53 +01:00
|
|
|
* singleton actor is terminated.
|
|
|
|
|
* Note that [[akka.actor.PoisonPill]] is a perfectly fine
|
|
|
|
|
* `terminationMessage` if you only need to stop the actor.
|
|
|
|
|
*
|
2015-04-29 18:23:45 +02:00
|
|
|
* @param settings see [[ClusterSingletonManagerSettings]]
|
2013-01-14 14:09:53 +01:00
|
|
|
*/
|
|
|
|
|
class ClusterSingletonManager(
|
2013-09-10 13:35:51 +02:00
|
|
|
singletonProps: Props,
|
2013-01-14 14:09:53 +01:00
|
|
|
terminationMessage: Any,
|
2015-04-29 18:23:45 +02:00
|
|
|
settings: ClusterSingletonManagerSettings)
|
2013-01-14 14:09:53 +01:00
|
|
|
extends Actor with FSM[ClusterSingletonManager.State, ClusterSingletonManager.Data] {
|
|
|
|
|
|
|
|
|
|
import ClusterSingletonManager._
|
|
|
|
|
import ClusterSingletonManager.Internal._
|
2013-04-28 22:05:40 +02:00
|
|
|
import ClusterSingletonManager.Internal.OldestChangedBuffer._
|
2015-04-29 18:23:45 +02:00
|
|
|
import settings._
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
val cluster = Cluster(context.system)
|
|
|
|
|
val selfAddressOption = Some(cluster.selfAddress)
|
2013-05-23 13:36:35 +02:00
|
|
|
import cluster.settings.LogInfo
|
2013-03-14 20:32:43 +01:00
|
|
|
|
2013-04-06 16:22:30 +02:00
|
|
|
require(role.forall(cluster.selfRoles.contains),
|
|
|
|
|
s"This cluster member [${cluster.selfAddress}] doesn't have the role [$role]")
|
2013-03-14 20:32:43 +01:00
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
// started when when self member is Up
|
2013-04-28 22:05:40 +02:00
|
|
|
var oldestChangedBuffer: ActorRef = _
|
2013-01-14 14:09:53 +01:00
|
|
|
// Previous GetNext request delivered event and new GetNext is to be sent
|
2013-04-28 22:05:40 +02:00
|
|
|
var oldestChangedReceived = true
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-05-09 09:49:59 +02:00
|
|
|
var selfExited = false
|
|
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
// keep track of previously removed members
|
|
|
|
|
var removed = Map.empty[Address, Deadline]
|
|
|
|
|
|
|
|
|
|
def addRemoved(address: Address): Unit =
|
|
|
|
|
removed += address -> (Deadline.now + 15.minutes)
|
|
|
|
|
|
|
|
|
|
def cleanupOverdueNotMemberAnyMore(): Unit = {
|
|
|
|
|
removed = removed filter { case (address, deadline) ⇒ deadline.hasTimeLeft }
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def logInfo(message: String): Unit =
|
2013-05-23 13:36:35 +02:00
|
|
|
if (LogInfo) log.info(message)
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
def logInfo(template: String, arg1: Any): Unit =
|
2013-05-23 13:36:35 +02:00
|
|
|
if (LogInfo) log.info(template, arg1)
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
def logInfo(template: String, arg1: Any, arg2: Any): Unit =
|
2013-05-23 13:36:35 +02:00
|
|
|
if (LogInfo) log.info(template, arg1, arg2)
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
super.preStart()
|
|
|
|
|
require(!cluster.isTerminated, "Cluster node must not be terminated")
|
|
|
|
|
|
|
|
|
|
// subscribe to cluster changes, re-subscribe when restart
|
2014-01-08 14:14:48 +01:00
|
|
|
cluster.subscribe(self, classOf[MemberExited], classOf[MemberRemoved])
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
setTimer(CleanupTimer, Cleanup, 1.minute, repeat = true)
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
// defer subscription to avoid some jitter when
|
2013-01-14 14:09:53 +01:00
|
|
|
// starting/joining several nodes at the same time
|
2013-04-28 22:05:40 +02:00
|
|
|
cluster.registerOnMemberUp(self ! StartOldestChangedBuffer)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
cancelTimer(CleanupTimer)
|
|
|
|
|
cluster.unsubscribe(self)
|
|
|
|
|
super.postStop()
|
|
|
|
|
}
|
|
|
|
|
|
2013-03-26 18:17:50 +01:00
|
|
|
def peer(at: Address): ActorSelection = context.actorSelection(self.path.toStringWithAddress(at))
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
def getNextOldestChanged(): Unit =
|
|
|
|
|
if (oldestChangedReceived) {
|
|
|
|
|
oldestChangedReceived = false
|
|
|
|
|
oldestChangedBuffer ! GetNext
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
startWith(Start, Uninitialized)
|
|
|
|
|
|
|
|
|
|
when(Start) {
|
2013-04-28 22:05:40 +02:00
|
|
|
case Event(StartOldestChangedBuffer, _) ⇒
|
|
|
|
|
oldestChangedBuffer = context.actorOf(Props(classOf[OldestChangedBuffer], role).
|
2013-04-17 22:14:19 +02:00
|
|
|
withDispatcher(context.props.dispatcher))
|
2013-04-28 22:05:40 +02:00
|
|
|
getNextOldestChanged()
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
|
|
|
|
|
2015-02-09 15:34:58 +01:00
|
|
|
case Event(InitialOldestState(oldestOption, safeToBeOldest), _) ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
oldestChangedReceived = true
|
2015-02-09 15:34:58 +01:00
|
|
|
if (oldestOption == selfAddressOption && safeToBeOldest)
|
|
|
|
|
// oldest immediately
|
2013-09-10 13:35:51 +02:00
|
|
|
gotoOldest()
|
2013-04-28 22:05:40 +02:00
|
|
|
else if (oldestOption == selfAddressOption)
|
|
|
|
|
goto(BecomingOldest) using BecomingOldestData(None)
|
2013-01-14 14:09:53 +01:00
|
|
|
else
|
2013-04-28 22:05:40 +02:00
|
|
|
goto(Younger) using YoungerData(oldestOption)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
when(Younger) {
|
|
|
|
|
case Event(OldestChanged(oldestOption), YoungerData(previousOldestOption)) ⇒
|
|
|
|
|
oldestChangedReceived = true
|
|
|
|
|
if (oldestOption == selfAddressOption) {
|
|
|
|
|
logInfo("Younger observed OldestChanged: [{} -> myself]", previousOldestOption)
|
|
|
|
|
previousOldestOption match {
|
2013-09-10 13:35:51 +02:00
|
|
|
case None ⇒ gotoOldest()
|
|
|
|
|
case Some(prev) if removed.contains(prev) ⇒ gotoOldest()
|
2013-01-14 14:09:53 +01:00
|
|
|
case Some(prev) ⇒
|
|
|
|
|
peer(prev) ! HandOverToMe
|
2013-04-28 22:05:40 +02:00
|
|
|
goto(BecomingOldest) using BecomingOldestData(previousOldestOption)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
} else {
|
2013-04-28 22:05:40 +02:00
|
|
|
logInfo("Younger observed OldestChanged: [{} -> {}]", previousOldestOption, oldestOption)
|
|
|
|
|
getNextOldestChanged()
|
|
|
|
|
stay using YoungerData(oldestOption)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2013-05-23 11:09:32 +02:00
|
|
|
case Event(MemberRemoved(m, _), YoungerData(Some(previousOldest))) if m.address == previousOldest ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
logInfo("Previous oldest removed [{}]", m.address)
|
2013-03-05 21:05:11 +01:00
|
|
|
addRemoved(m.address)
|
2013-04-28 22:05:40 +02:00
|
|
|
// transition when OldestChanged
|
|
|
|
|
stay using YoungerData(None)
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-05-23 11:09:32 +02:00
|
|
|
case Event(MemberRemoved(m, _), _) if m.address == cluster.selfAddress ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
logInfo("Self removed, stopping ClusterSingletonManager")
|
|
|
|
|
stop()
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
when(BecomingOldest) {
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
case Event(HandOverInProgress, _) ⇒
|
2013-01-28 08:47:52 +01:00
|
|
|
// confirmation that the hand-over process has started
|
2014-01-16 15:16:35 +01:00
|
|
|
logInfo("Hand-over in progress at [{}]", sender().path.address)
|
2013-01-14 14:09:53 +01:00
|
|
|
cancelTimer(HandOverRetryTimer)
|
|
|
|
|
stay
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(HandOverDone, BecomingOldestData(Some(previousOldest))) ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
if (sender().path.address == previousOldest)
|
2013-09-10 13:35:51 +02:00
|
|
|
gotoOldest()
|
2013-01-14 14:09:53 +01:00
|
|
|
else {
|
2013-04-28 22:05:40 +02:00
|
|
|
logInfo("Ignoring HandOverDone in BecomingOldest from [{}]. Expected previous oldest [{}]",
|
2014-01-16 15:16:35 +01:00
|
|
|
sender().path.address, previousOldest)
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-23 11:09:32 +02:00
|
|
|
case Event(MemberRemoved(m, _), BecomingOldestData(Some(previousOldest))) if m.address == previousOldest ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
logInfo("Previous oldest [{}] removed", previousOldest)
|
2013-03-05 21:05:11 +01:00
|
|
|
addRemoved(m.address)
|
2013-03-08 09:39:48 +01:00
|
|
|
stay
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
case Event(TakeOverFromMe, BecomingOldestData(None)) ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
sender() ! HandOverToMe
|
|
|
|
|
stay using BecomingOldestData(Some(sender().path.address))
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
case Event(TakeOverFromMe, BecomingOldestData(Some(previousOldest))) ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
if (previousOldest == sender().path.address) sender() ! HandOverToMe
|
2013-04-28 22:05:40 +02:00
|
|
|
else logInfo("Ignoring TakeOver request in BecomingOldest from [{}]. Expected previous oldest [{}]",
|
2014-01-16 15:16:35 +01:00
|
|
|
sender().path.address, previousOldest)
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
case Event(HandOverRetry(count), BecomingOldestData(previousOldestOption)) ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
if (count <= maxHandOverRetries) {
|
2013-04-28 22:05:40 +02:00
|
|
|
logInfo("Retry [{}], sending HandOverToMe to [{}]", count, previousOldestOption)
|
|
|
|
|
previousOldestOption foreach { peer(_) ! HandOverToMe }
|
2013-01-14 14:09:53 +01:00
|
|
|
setTimer(HandOverRetryTimer, HandOverRetry(count + 1), retryInterval, repeat = false)
|
2013-03-27 17:47:56 +01:00
|
|
|
stay()
|
2013-04-28 22:05:40 +02:00
|
|
|
} else if (previousOldestOption forall removed.contains) {
|
|
|
|
|
// can't send HandOverToMe, previousOldest unknown for new node (or restart)
|
|
|
|
|
// previous oldest might be down or removed, so no TakeOverFromMe message is received
|
|
|
|
|
logInfo("Timeout in BecomingOldest. Previous oldest unknown, removed and no TakeOver request.")
|
2013-09-10 13:35:51 +02:00
|
|
|
gotoOldest()
|
2013-01-14 14:09:53 +01:00
|
|
|
} else
|
|
|
|
|
throw new ClusterSingletonManagerIsStuck(
|
2013-04-28 22:05:40 +02:00
|
|
|
s"Becoming singleton oldest was stuck because previous oldest [${previousOldestOption}] is unresponsive")
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
def gotoOldest(): State = {
|
|
|
|
|
val singleton = context watch context.actorOf(singletonProps, singletonName)
|
2015-06-04 21:21:37 +02:00
|
|
|
logInfo("Singleton manager starting singleton actor [{}]", singleton.path)
|
2013-04-28 22:05:40 +02:00
|
|
|
goto(Oldest) using OldestData(singleton)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
when(Oldest) {
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(OldestChanged(oldestOption), OldestData(singleton, singletonTerminated)) ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
oldestChangedReceived = true
|
|
|
|
|
logInfo("Oldest observed OldestChanged: [{} -> {}]", cluster.selfAddress, oldestOption)
|
|
|
|
|
oldestOption match {
|
2013-01-14 14:09:53 +01:00
|
|
|
case Some(a) if a == cluster.selfAddress ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
// already oldest
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
2013-05-29 09:11:43 +02:00
|
|
|
case Some(a) if !selfExited && removed.contains(a) ⇒
|
2013-09-10 13:35:51 +02:00
|
|
|
gotoHandingOver(singleton, singletonTerminated, None)
|
2013-01-14 14:09:53 +01:00
|
|
|
case Some(a) ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
// send TakeOver request in case the new oldest doesn't know previous oldest
|
2013-02-07 14:25:29 +01:00
|
|
|
peer(a) ! TakeOverFromMe
|
|
|
|
|
setTimer(TakeOverRetryTimer, TakeOverRetry(1), retryInterval, repeat = false)
|
2013-09-10 13:35:51 +02:00
|
|
|
goto(WasOldest) using WasOldestData(singleton, singletonTerminated, newOldestOption = Some(a))
|
2013-02-07 14:25:29 +01:00
|
|
|
case None ⇒
|
2013-04-28 22:05:40 +02:00
|
|
|
// new oldest will initiate the hand-over
|
2013-02-07 14:25:29 +01:00
|
|
|
setTimer(TakeOverRetryTimer, TakeOverRetry(1), retryInterval, repeat = false)
|
2013-09-10 13:35:51 +02:00
|
|
|
goto(WasOldest) using WasOldestData(singleton, singletonTerminated, newOldestOption = None)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(HandOverToMe, OldestData(singleton, singletonTerminated)) ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
gotoHandingOver(singleton, singletonTerminated, Some(sender()))
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(Terminated(ref), d @ OldestData(singleton, _)) if ref == singleton ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
stay using d.copy(singletonTerminated = true)
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
when(WasOldest) {
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(TakeOverRetry(count), WasOldestData(_, _, newOldestOption)) ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
if (count <= maxTakeOverRetries) {
|
2013-04-28 22:05:40 +02:00
|
|
|
logInfo("Retry [{}], sending TakeOverFromMe to [{}]", count, newOldestOption)
|
|
|
|
|
newOldestOption foreach { peer(_) ! TakeOverFromMe }
|
2013-02-07 14:25:29 +01:00
|
|
|
setTimer(TakeOverRetryTimer, TakeOverRetry(count + 1), retryInterval, repeat = false)
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
|
|
|
|
} else
|
2013-04-28 22:05:40 +02:00
|
|
|
throw new ClusterSingletonManagerIsStuck(s"Expected hand-over to [${newOldestOption}] never occured")
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(HandOverToMe, WasOldestData(singleton, singletonTerminated, _)) ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
gotoHandingOver(singleton, singletonTerminated, Some(sender()))
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(MemberRemoved(m, _), WasOldestData(singleton, singletonTerminated, Some(newOldest))) if !selfExited && m.address == newOldest ⇒
|
2013-03-05 21:05:11 +01:00
|
|
|
addRemoved(m.address)
|
2013-09-10 13:35:51 +02:00
|
|
|
gotoHandingOver(singleton, singletonTerminated, None)
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
case Event(Terminated(ref), d @ WasOldestData(singleton, _, _)) if ref == singleton ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
stay using d.copy(singletonTerminated = true)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
def gotoHandingOver(singleton: ActorRef, singletonTerminated: Boolean, handOverTo: Option[ActorRef]): State = {
|
2013-01-14 14:09:53 +01:00
|
|
|
if (singletonTerminated) {
|
2013-09-10 13:35:51 +02:00
|
|
|
handOverDone(handOverTo)
|
2013-01-14 14:09:53 +01:00
|
|
|
} else {
|
|
|
|
|
handOverTo foreach { _ ! HandOverInProgress }
|
|
|
|
|
singleton ! terminationMessage
|
2013-09-10 13:35:51 +02:00
|
|
|
goto(HandingOver) using HandingOverData(singleton, handOverTo)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
when(HandingOver) {
|
2013-09-10 13:35:51 +02:00
|
|
|
case (Event(Terminated(ref), HandingOverData(singleton, handOverTo))) if ref == singleton ⇒
|
|
|
|
|
handOverDone(handOverTo)
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2014-01-16 15:16:35 +01:00
|
|
|
case Event(HandOverToMe, d @ HandingOverData(singleton, handOverTo)) if handOverTo == Some(sender()) ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
// retry
|
2014-01-16 15:16:35 +01:00
|
|
|
sender() ! HandOverInProgress
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
def handOverDone(handOverTo: Option[ActorRef]): State = {
|
2013-04-28 22:05:40 +02:00
|
|
|
val newOldest = handOverTo.map(_.path.address)
|
|
|
|
|
logInfo("Singleton terminated, hand-over done [{} -> {}]", cluster.selfAddress, newOldest)
|
2013-09-10 13:35:51 +02:00
|
|
|
handOverTo foreach { _ ! HandOverDone }
|
2015-02-09 15:34:58 +01:00
|
|
|
if (removed.contains(cluster.selfAddress)) {
|
|
|
|
|
logInfo("Self removed, stopping ClusterSingletonManager")
|
|
|
|
|
stop()
|
|
|
|
|
} else if (selfExited)
|
2013-05-09 09:49:59 +02:00
|
|
|
goto(End) using EndData
|
|
|
|
|
else
|
|
|
|
|
goto(Younger) using YoungerData(newOldest)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
when(End) {
|
2013-05-23 11:09:32 +02:00
|
|
|
case Event(MemberRemoved(m, _), _) if m.address == cluster.selfAddress ⇒
|
2013-05-09 09:49:59 +02:00
|
|
|
logInfo("Self removed, stopping ClusterSingletonManager")
|
|
|
|
|
stop()
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
whenUnhandled {
|
|
|
|
|
case Event(_: CurrentClusterState, _) ⇒ stay
|
2013-05-09 09:49:59 +02:00
|
|
|
case Event(MemberExited(m), _) ⇒
|
|
|
|
|
if (m.address == cluster.selfAddress) {
|
|
|
|
|
selfExited = true
|
|
|
|
|
logInfo("Exited [{}]", m.address)
|
|
|
|
|
}
|
|
|
|
|
stay
|
2015-02-09 15:34:58 +01:00
|
|
|
case Event(MemberRemoved(m, _), _) if m.address == cluster.selfAddress && !selfExited ⇒
|
|
|
|
|
logInfo("Self removed, stopping ClusterSingletonManager")
|
|
|
|
|
stop()
|
2013-05-23 11:09:32 +02:00
|
|
|
case Event(MemberRemoved(m, _), _) ⇒
|
2013-05-09 09:49:59 +02:00
|
|
|
if (!selfExited) logInfo("Member removed [{}]", m.address)
|
2013-01-14 14:09:53 +01:00
|
|
|
addRemoved(m.address)
|
|
|
|
|
stay
|
|
|
|
|
case Event(TakeOverFromMe, _) ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
logInfo("Ignoring TakeOver request in [{}] from [{}].", stateName, sender().path.address)
|
2013-01-14 14:09:53 +01:00
|
|
|
stay
|
|
|
|
|
case Event(Cleanup, _) ⇒
|
|
|
|
|
cleanupOverdueNotMemberAnyMore()
|
|
|
|
|
stay
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTransition {
|
|
|
|
|
case from -> to ⇒ logInfo("ClusterSingletonManager state change [{} -> {}]", from, to)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTransition {
|
2013-04-28 22:05:40 +02:00
|
|
|
case _ -> BecomingOldest ⇒ setTimer(HandOverRetryTimer, HandOverRetry(1), retryInterval, repeat = false)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTransition {
|
2013-04-28 22:05:40 +02:00
|
|
|
case BecomingOldest -> _ ⇒ cancelTimer(HandOverRetryTimer)
|
|
|
|
|
case WasOldest -> _ ⇒ cancelTimer(TakeOverRetryTimer)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTransition {
|
2013-04-28 22:05:40 +02:00
|
|
|
case _ -> (Younger | Oldest) ⇒ getNextOldestChanged()
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTransition {
|
2013-05-09 09:49:59 +02:00
|
|
|
case _ -> (Younger | End) if removed.contains(cluster.selfAddress) ⇒
|
2013-01-14 14:09:53 +01:00
|
|
|
logInfo("Self removed, stopping ClusterSingletonManager")
|
2013-05-09 09:49:59 +02:00
|
|
|
// note that FSM.stop() can't be used in onTransition
|
|
|
|
|
context.stop(self)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2014-05-07 11:09:53 +05:30
|
|
|
}
|