2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2017-01-04 17:37:10 +01:00
|
|
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
|
|
|
|
package akka.cluster
|
|
|
|
|
|
2013-08-23 14:39:21 +02:00
|
|
|
import akka.actor._
|
2017-07-10 13:01:06 +01:00
|
|
|
import akka.annotation.InternalApi
|
2013-02-11 10:40:01 +01:00
|
|
|
import akka.actor.SupervisorStrategy.Stop
|
2012-08-14 10:58:30 +02:00
|
|
|
import akka.cluster.MemberStatus._
|
|
|
|
|
import akka.cluster.ClusterEvent._
|
2017-07-04 09:09:40 +01:00
|
|
|
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
2017-07-10 13:01:06 +01:00
|
|
|
import akka.Done
|
2016-12-01 18:49:38 +01:00
|
|
|
import akka.pattern.ask
|
2017-07-10 13:01:06 +01:00
|
|
|
import akka.remote.QuarantinedEvent
|
2016-12-01 18:49:38 +01:00
|
|
|
import akka.util.Timeout
|
2017-07-04 09:09:40 +01:00
|
|
|
|
2017-07-10 13:01:06 +01:00
|
|
|
import scala.collection.immutable
|
|
|
|
|
import scala.concurrent.duration._
|
2016-12-01 18:49:38 +01:00
|
|
|
import scala.concurrent.Future
|
|
|
|
|
import scala.concurrent.Promise
|
2017-07-10 13:01:06 +01:00
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
import language.existentials
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Base trait for all cluster messages. All ClusterMessage's are serializable.
|
|
|
|
|
*/
|
|
|
|
|
trait ClusterMessage extends Serializable
|
|
|
|
|
|
|
|
|
|
/**
|
2013-04-11 09:18:12 +02:00
|
|
|
* INTERNAL API
|
|
|
|
|
* Cluster commands sent by the USER via
|
|
|
|
|
* [[akka.cluster.Cluster]] extension
|
|
|
|
|
* or JMX.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-04-11 09:18:12 +02:00
|
|
|
private[cluster] object ClusterUserAction {
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
2013-04-11 09:18:12 +02:00
|
|
|
* Command to initiate join another node (represented by `address`).
|
|
|
|
|
* Join will be sent to the other node.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class JoinTo(address: Address)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Command to leave the cluster.
|
|
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Leave(address: Address) extends ClusterMessage
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Command to mark node as temporary down.
|
|
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Down(address: Address) extends ClusterMessage
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[cluster] object InternalClusterAction {
|
|
|
|
|
|
|
|
|
|
/**
|
2013-04-11 09:18:12 +02:00
|
|
|
* Command to join the cluster. Sent when a node wants to join another node (the receiver).
|
2016-04-11 10:33:02 +02:00
|
|
|
*
|
2013-04-11 09:18:12 +02:00
|
|
|
* @param node the node that wants to join the cluster
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-04-11 09:18:12 +02:00
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Join(node: UniqueAddress, roles: Set[String]) extends ClusterMessage
|
2013-04-11 09:18:12 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Reply to Join
|
2016-04-11 10:33:02 +02:00
|
|
|
*
|
2013-04-11 09:18:12 +02:00
|
|
|
* @param from the sender node in the cluster, i.e. the node that received the Join command
|
|
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Welcome(from: UniqueAddress, gossip: Gossip) extends ClusterMessage
|
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.
|
|
|
|
|
*/
|
2014-03-07 13:20:01 +01:00
|
|
|
final 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
|
|
|
*/
|
2016-08-09 17:22:31 +02:00
|
|
|
case object JoinSeedNode extends DeadLetterSuppression
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-05-15 16:53:24 +02:00
|
|
|
* see JoinSeedNode
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2016-08-09 17:22:31 +02:00
|
|
|
case object InitJoin extends ClusterMessage with DeadLetterSuppression
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-05-15 16:53:24 +02:00
|
|
|
* see JoinSeedNode
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2016-08-09 17:22:31 +02:00
|
|
|
final case class InitJoinAck(address: Address) extends ClusterMessage with DeadLetterSuppression
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-02-17 17:35:43 +01:00
|
|
|
/**
|
2015-05-15 16:53:24 +02:00
|
|
|
* see JoinSeedNode
|
2013-02-17 17:35:43 +01:00
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2016-08-09 17:22:31 +02:00
|
|
|
final case class InitJoinNack(address: Address) extends ClusterMessage with DeadLetterSuppression
|
2013-02-17 17:35:43 +01:00
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
final case class ExitingConfirmed(node: UniqueAddress) extends ClusterMessage with DeadLetterSuppression
|
|
|
|
|
|
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
|
|
|
|
2013-10-14 14:50:57 +02:00
|
|
|
case object GossipSpeedupTick extends Tick
|
|
|
|
|
|
2012-08-14 13:55:22 +02:00
|
|
|
case object ReapUnreachableTick extends Tick
|
2012-07-05 13:55:08 +02:00
|
|
|
|
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
|
|
|
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class SendGossipTo(address: Address)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
case object GetClusterCoreRef
|
|
|
|
|
|
2012-12-10 08:46:25 +01:00
|
|
|
/**
|
2015-06-02 21:01:00 -07:00
|
|
|
* Command to [[akka.cluster.ClusterDaemon]] to create a
|
2015-04-17 17:28:37 +08:00
|
|
|
* [[akka.cluster.OnMemberStatusChangedListener]].
|
2012-12-10 08:46:25 +01:00
|
|
|
*/
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class AddOnMemberUpListener(callback: Runnable) extends NoSerializationVerificationNeeded
|
2012-12-10 08:46:25 +01:00
|
|
|
|
2015-04-17 17:28:37 +08:00
|
|
|
final case class AddOnMemberRemovedListener(callback: Runnable) extends NoSerializationVerificationNeeded
|
|
|
|
|
|
2012-08-19 20:15:22 +02:00
|
|
|
sealed trait SubscriptionMessage
|
2016-12-01 18:49:38 +01:00
|
|
|
final case class Subscribe(subscriber: ActorRef, initialStateMode: SubscriptionInitialStateMode,
|
|
|
|
|
to: Set[Class[_]]) extends SubscriptionMessage
|
|
|
|
|
final case class Unsubscribe(subscriber: ActorRef, to: Option[Class[_]])
|
|
|
|
|
extends SubscriptionMessage with DeadLetterSuppression
|
2012-09-12 09:23:02 +02:00
|
|
|
/**
|
2014-03-12 11:38:09 +01:00
|
|
|
* @param receiver [[akka.cluster.ClusterEvent.CurrentClusterState]] will be sent to the `receiver`
|
2012-09-12 09:23:02 +02:00
|
|
|
*/
|
2014-03-12 11:38:09 +01:00
|
|
|
final case class SendCurrentClusterState(receiver: ActorRef) extends SubscriptionMessage
|
2012-08-19 20:15:22 +02:00
|
|
|
|
2012-11-27 18:07:37 +01:00
|
|
|
sealed trait PublishMessage
|
2017-07-04 21:58:03 +02:00
|
|
|
final case class PublishChanges(state: MembershipState) extends PublishMessage
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class PublishEvent(event: ClusterDomainEvent) extends PublishMessage
|
2016-12-01 18:49:38 +01:00
|
|
|
|
|
|
|
|
final case object ExitingCompleted
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
*
|
|
|
|
|
* Supervisor managing the different Cluster daemons.
|
|
|
|
|
*/
|
2013-04-26 12:18:01 +02:00
|
|
|
private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Actor with ActorLogging
|
|
|
|
|
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
2013-02-11 10:40:01 +01:00
|
|
|
import InternalClusterAction._
|
2015-08-27 08:58:52 +02:00
|
|
|
// Important - don't use Cluster(context.system) in constructor because that would
|
2012-09-06 21:48:40 +02:00
|
|
|
// cause deadlock. The Cluster extension is currently being created and is waiting
|
|
|
|
|
// for response from GetClusterCoreRef in its constructor.
|
2015-08-27 08:58:52 +02:00
|
|
|
// Child actors are therefore created when GetClusterCoreRef is received
|
|
|
|
|
var coreSupervisor: Option[ActorRef] = None
|
|
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
val clusterShutdown = Promise[Done]()
|
|
|
|
|
val coordShutdown = CoordinatedShutdown(context.system)
|
|
|
|
|
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterLeave, "leave") {
|
|
|
|
|
val sys = context.system
|
|
|
|
|
() ⇒
|
2017-11-23 08:48:38 +01:00
|
|
|
if (Cluster(sys).isTerminated || Cluster(sys).selfMember.status == Down)
|
2016-12-01 18:49:38 +01:00
|
|
|
Future.successful(Done)
|
|
|
|
|
else {
|
|
|
|
|
implicit val timeout = Timeout(coordShutdown.timeout(CoordinatedShutdown.PhaseClusterLeave))
|
|
|
|
|
self.ask(CoordinatedShutdownLeave.LeaveReq).mapTo[Done]
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterShutdown, "wait-shutdown") { () ⇒
|
|
|
|
|
clusterShutdown.future
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
clusterShutdown.trySuccess(Done)
|
|
|
|
|
if (Cluster(context.system).settings.RunCoordinatedShutdownWhenDown) {
|
2017-12-04 12:22:59 +01:00
|
|
|
// if it was stopped due to leaving CoordinatedShutdown was started earlier
|
|
|
|
|
coordShutdown.run(CoordinatedShutdown.ClusterDowningReason)
|
2016-12-01 18:49:38 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-08-27 08:58:52 +02:00
|
|
|
def createChildren(): Unit = {
|
|
|
|
|
coreSupervisor = Some(context.actorOf(Props[ClusterCoreSupervisor].
|
|
|
|
|
withDispatcher(context.props.dispatcher), name = "core"))
|
|
|
|
|
context.actorOf(Props[ClusterHeartbeatReceiver].
|
|
|
|
|
withDispatcher(context.props.dispatcher), name = "heartbeatReceiver")
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
def receive = {
|
2015-08-27 08:58:52 +02:00
|
|
|
case msg: GetClusterCoreRef.type ⇒
|
|
|
|
|
if (coreSupervisor.isEmpty)
|
|
|
|
|
createChildren()
|
|
|
|
|
coreSupervisor.foreach(_ forward msg)
|
2013-02-11 10:40:01 +01:00
|
|
|
case AddOnMemberUpListener(code) ⇒
|
2015-04-17 17:28:37 +08:00
|
|
|
context.actorOf(Props(classOf[OnMemberStatusChangedListener], code, Up).withDeploy(Deploy.local))
|
|
|
|
|
case AddOnMemberRemovedListener(code) ⇒
|
|
|
|
|
context.actorOf(Props(classOf[OnMemberStatusChangedListener], code, Removed).withDeploy(Deploy.local))
|
2016-12-01 18:49:38 +01:00
|
|
|
case CoordinatedShutdownLeave.LeaveReq ⇒
|
|
|
|
|
val ref = context.actorOf(CoordinatedShutdownLeave.props().withDispatcher(context.props.dispatcher))
|
|
|
|
|
// forward the ask request
|
|
|
|
|
ref.forward(CoordinatedShutdownLeave.LeaveReq)
|
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.
|
|
|
|
|
*/
|
2013-04-26 12:18:01 +02:00
|
|
|
private[cluster] final class ClusterCoreSupervisor extends Actor with ActorLogging
|
|
|
|
|
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
2013-02-11 10:40:01 +01:00
|
|
|
|
2015-08-27 08:58:52 +02:00
|
|
|
// Important - don't use Cluster(context.system) in constructor because that would
|
|
|
|
|
// cause deadlock. The Cluster extension is currently being created and is waiting
|
|
|
|
|
// for response from GetClusterCoreRef in its constructor.
|
|
|
|
|
// Child actors are therefore created when GetClusterCoreRef is received
|
2013-02-11 10:40:01 +01:00
|
|
|
|
2015-08-27 08:58:52 +02:00
|
|
|
var coreDaemon: Option[ActorRef] = None
|
|
|
|
|
|
|
|
|
|
def createChildren(): Unit = {
|
|
|
|
|
val publisher = context.actorOf(Props[ClusterDomainEventPublisher].
|
|
|
|
|
withDispatcher(context.props.dispatcher), name = "publisher")
|
|
|
|
|
coreDaemon = Some(context.watch(context.actorOf(Props(classOf[ClusterCoreDaemon], publisher).
|
|
|
|
|
withDispatcher(context.props.dispatcher), name = "daemon")))
|
|
|
|
|
}
|
2013-02-11 10:40:01 +01:00
|
|
|
|
|
|
|
|
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 = {
|
2015-08-27 08:58:52 +02:00
|
|
|
case InternalClusterAction.GetClusterCoreRef ⇒
|
|
|
|
|
if (coreDaemon.isEmpty)
|
|
|
|
|
createChildren()
|
|
|
|
|
coreDaemon.foreach(sender() ! _)
|
2013-02-11 10:40:01 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
*/
|
2017-07-04 09:09:40 +01:00
|
|
|
@InternalApi
|
|
|
|
|
private[cluster] object ClusterCoreDaemon {
|
|
|
|
|
val NumberOfGossipsBeforeShutdownWhenLeaderExits = 5
|
|
|
|
|
val MaxGossipsBeforeShuttingDownMyself = 5
|
2017-07-04 21:58:03 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API.
|
|
|
|
|
*/
|
|
|
|
|
@InternalApi
|
2013-06-13 15:43:37 -04:00
|
|
|
private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with ActorLogging
|
2013-04-26 12:18:01 +02:00
|
|
|
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
2012-07-05 13:55:08 +02:00
|
|
|
import InternalClusterAction._
|
2017-07-04 09:09:40 +01:00
|
|
|
import ClusterCoreDaemon._
|
2017-07-04 21:58:03 +02:00
|
|
|
import MembershipState._
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2012-09-06 21:48:40 +02:00
|
|
|
val cluster = Cluster(context.system)
|
2017-08-22 15:02:27 +02:00
|
|
|
import cluster.{ selfAddress, selfRoles, scheduler, failureDetector, crossDcFailureDetector }
|
2014-03-12 11:47:43 +01:00
|
|
|
import cluster.settings._
|
2013-05-23 13:36:35 +02:00
|
|
|
import cluster.InfoLogger._
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-07 13:17:41 +02:00
|
|
|
val selfDc = cluster.selfDataCenter
|
2013-06-13 15:43:37 -04:00
|
|
|
|
|
|
|
|
protected def selfUniqueAddress = cluster.selfUniqueAddress
|
2013-05-09 09:49:59 +02:00
|
|
|
|
2017-07-10 13:01:06 +01:00
|
|
|
val vclockNode = VectorClock.Node(Gossip.vclockName(selfUniqueAddress))
|
2017-07-07 13:19:10 +01:00
|
|
|
val gossipTargetSelector = new GossipTargetSelector(
|
|
|
|
|
ReduceGossipDifferentViewProbability,
|
|
|
|
|
cluster.settings.MultiDataCenter.CrossDcGossipProbability)
|
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
|
2017-07-07 13:19:10 +01:00
|
|
|
var membershipState = MembershipState(
|
|
|
|
|
Gossip.empty,
|
|
|
|
|
cluster.selfUniqueAddress,
|
2017-07-12 11:47:32 +01:00
|
|
|
cluster.settings.SelfDataCenter,
|
2017-07-07 13:19:10 +01:00
|
|
|
cluster.settings.MultiDataCenter.CrossDcConnections)
|
|
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
def latestGossip: Gossip = membershipState.latestGossip
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
val statsEnabled = PublishStatsInterval.isFinite
|
|
|
|
|
var gossipStats = GossipStats()
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2015-05-04 08:35:46 +02:00
|
|
|
var seedNodes = SeedNodes
|
2013-02-17 17:35:43 +01:00
|
|
|
var seedNodeProcess: Option[ActorRef] = None
|
2015-01-30 14:30:16 +01:00
|
|
|
var seedNodeProcessCounter = 0 // for unique names
|
2017-08-21 10:49:56 +02:00
|
|
|
var joinSeedNodesDeadline: Option[Deadline] = None
|
2015-01-30 14:30:16 +01:00
|
|
|
var leaderActionCounter = 0
|
2013-02-17 17:35:43 +01:00
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
var exitingTasksInProgress = false
|
|
|
|
|
val selfExiting = Promise[Done]()
|
|
|
|
|
val coordShutdown = CoordinatedShutdown(context.system)
|
|
|
|
|
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterExiting, "wait-exiting") { () ⇒
|
2017-04-11 21:48:51 +02:00
|
|
|
if (latestGossip.members.isEmpty)
|
|
|
|
|
Future.successful(Done) // not joined yet
|
|
|
|
|
else
|
|
|
|
|
selfExiting.future
|
2016-12-01 18:49:38 +01:00
|
|
|
}
|
|
|
|
|
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterExitingDone, "exiting-completed") {
|
|
|
|
|
val sys = context.system
|
|
|
|
|
() ⇒
|
2017-11-23 08:48:38 +01:00
|
|
|
if (Cluster(sys).isTerminated || Cluster(sys).selfMember.status == Down)
|
2016-12-01 18:49:38 +01:00
|
|
|
Future.successful(Done)
|
|
|
|
|
else {
|
|
|
|
|
implicit val timeout = Timeout(coordShutdown.timeout(CoordinatedShutdown.PhaseClusterExitingDone))
|
|
|
|
|
self.ask(ExitingCompleted).mapTo[Done]
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
var exitingConfirmed = Set.empty[UniqueAddress]
|
|
|
|
|
|
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-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
|
2016-06-02 14:06:57 +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)
|
2016-06-02 14:06:57 +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)
|
2016-06-02 14:06:57 +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
|
2013-04-11 09:18:12 +02:00
|
|
|
val publishStatsTask: Option[Cancellable] = PublishStatsInterval match {
|
2013-05-16 17:00:04 +02:00
|
|
|
case Duration.Zero | _: Duration.Infinite ⇒ None
|
2013-04-11 09:18:12 +02:00
|
|
|
case d: FiniteDuration ⇒
|
|
|
|
|
Some(scheduler.schedule(PeriodicTasksInitialDelay.max(d), d, self, PublishStatsTick))
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
context.system.eventStream.subscribe(self, classOf[QuarantinedEvent])
|
2013-09-11 16:09:51 +02:00
|
|
|
|
2016-04-11 10:33:02 +02:00
|
|
|
cluster.downingProvider.downingActorProps.foreach { props ⇒
|
|
|
|
|
val propsWithDispatcher =
|
|
|
|
|
if (props.dispatcher == Deploy.NoDispatcherGiven) props.withDispatcher(context.props.dispatcher)
|
|
|
|
|
else props
|
|
|
|
|
|
|
|
|
|
context.actorOf(propsWithDispatcher, name = "downingProvider")
|
2013-09-11 16:09:51 +02:00
|
|
|
}
|
|
|
|
|
|
2015-05-04 08:35:46 +02:00
|
|
|
if (seedNodes.isEmpty)
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("No seed-nodes configured, manual cluster join required")
|
2013-05-16 17:00:04 +02:00
|
|
|
else
|
2015-05-04 08:35:46 +02:00
|
|
|
self ! JoinSeedNodes(seedNodes)
|
2013-08-27 15:14:53 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
2013-08-27 15:14:53 +02:00
|
|
|
context.system.eventStream.unsubscribe(self)
|
2012-07-05 13:55:08 +02:00
|
|
|
gossipTask.cancel()
|
|
|
|
|
failureDetectorReaperTask.cancel()
|
|
|
|
|
leaderActionsTask.cancel()
|
2012-09-24 13:07:11 -06:00
|
|
|
publishStatsTask foreach { _.cancel() }
|
2016-12-01 18:49:38 +01:00
|
|
|
selfExiting.trySuccess(Done)
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
def uninitialized: Actor.Receive = ({
|
2017-05-19 07:20:29 -05:00
|
|
|
case InitJoin ⇒
|
|
|
|
|
logInfo("Received InitJoin message from [{}], but this node is not initialized yet", sender())
|
|
|
|
|
sender() ! InitJoinNack(selfAddress)
|
2017-08-21 10:49:56 +02:00
|
|
|
case ClusterUserAction.JoinTo(address) ⇒
|
|
|
|
|
join(address)
|
|
|
|
|
case JoinSeedNodes(newSeedNodes) ⇒
|
|
|
|
|
resetJoinSeedNodesDeadline()
|
|
|
|
|
joinSeedNodes(newSeedNodes)
|
|
|
|
|
case msg: SubscriptionMessage ⇒
|
|
|
|
|
publisher forward msg
|
|
|
|
|
case _: Tick ⇒
|
|
|
|
|
if (joinSeedNodesDeadline.exists(_.isOverdue))
|
|
|
|
|
joinSeedNodesWasUnsuccessful()
|
2016-12-01 18:49:38 +01:00
|
|
|
}: Actor.Receive).orElse(receiveExitingCompleted)
|
2013-04-11 09:18:12 +02:00
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
def tryingToJoin(joinWith: Address, deadline: Option[Deadline]): Actor.Receive = ({
|
2013-04-11 09:18:12 +02:00
|
|
|
case Welcome(from, gossip) ⇒ welcome(joinWith, from, gossip)
|
2017-05-19 07:20:29 -05:00
|
|
|
case InitJoin ⇒
|
|
|
|
|
logInfo("Received InitJoin message from [{}], but this node is not a member yet", sender())
|
|
|
|
|
sender() ! InitJoinNack(selfAddress)
|
2013-04-11 09:18:12 +02:00
|
|
|
case ClusterUserAction.JoinTo(address) ⇒
|
2013-09-17 14:20:29 +02:00
|
|
|
becomeUninitialized()
|
2013-04-11 09:18:12 +02:00
|
|
|
join(address)
|
2015-05-04 08:35:46 +02:00
|
|
|
case JoinSeedNodes(newSeedNodes) ⇒
|
2017-08-21 10:49:56 +02:00
|
|
|
resetJoinSeedNodesDeadline()
|
2013-09-17 14:20:29 +02:00
|
|
|
becomeUninitialized()
|
2015-05-04 08:35:46 +02:00
|
|
|
joinSeedNodes(newSeedNodes)
|
2012-08-19 20:15:22 +02:00
|
|
|
case msg: SubscriptionMessage ⇒ publisher forward msg
|
2013-04-11 09:18:12 +02:00
|
|
|
case _: Tick ⇒
|
2017-08-21 10:49:56 +02:00
|
|
|
if (joinSeedNodesDeadline.exists(_.isOverdue))
|
|
|
|
|
joinSeedNodesWasUnsuccessful()
|
|
|
|
|
else if (deadline.exists(_.isOverdue)) {
|
2015-05-04 08:35:46 +02:00
|
|
|
// join attempt failed, retry
|
2013-09-17 14:20:29 +02:00
|
|
|
becomeUninitialized()
|
2015-05-04 08:35:46 +02:00
|
|
|
if (seedNodes.nonEmpty) joinSeedNodes(seedNodes)
|
2013-04-11 09:18:12 +02:00
|
|
|
else join(joinWith)
|
|
|
|
|
}
|
2016-12-01 18:49:38 +01:00
|
|
|
}: Actor.Receive).orElse(receiveExitingCompleted)
|
2012-08-14 13:55:22 +02:00
|
|
|
|
2017-08-21 10:49:56 +02:00
|
|
|
private def resetJoinSeedNodesDeadline(): Unit = {
|
|
|
|
|
joinSeedNodesDeadline = ShutdownAfterUnsuccessfulJoinSeedNodes match {
|
|
|
|
|
case d: FiniteDuration ⇒ Some(Deadline.now + d)
|
|
|
|
|
case _ ⇒ None // off
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def joinSeedNodesWasUnsuccessful(): Unit = {
|
|
|
|
|
log.warning(
|
|
|
|
|
"Joining of seed-nodes [{}] was unsuccessful after configured " +
|
|
|
|
|
"shutdown-after-unsuccessful-join-seed-nodes [{}]. Running CoordinatedShutdown.",
|
|
|
|
|
seedNodes.mkString(", "), ShutdownAfterUnsuccessfulJoinSeedNodes)
|
|
|
|
|
joinSeedNodesDeadline = None
|
2017-12-04 12:22:59 +01:00
|
|
|
CoordinatedShutdown(context.system).run(CoordinatedShutdown.ClusterDowningReason)
|
2017-08-21 10:49:56 +02:00
|
|
|
}
|
|
|
|
|
|
2013-09-17 14:20:29 +02:00
|
|
|
def becomeUninitialized(): Unit = {
|
|
|
|
|
// make sure that join process is stopped
|
|
|
|
|
stopSeedNodeProcess()
|
|
|
|
|
context.become(uninitialized)
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-24 15:38:24 +02:00
|
|
|
def becomeInitialized(): Unit = {
|
|
|
|
|
// start heartbeatSender here, and not in constructor to make sure that
|
|
|
|
|
// heartbeating doesn't start before Welcome is received
|
2017-07-07 13:17:41 +02:00
|
|
|
val internalHeartbeatSenderProps = Props(new ClusterHeartbeatSender()).withDispatcher(UseDispatcher)
|
|
|
|
|
context.actorOf(internalHeartbeatSenderProps, name = "heartbeatSender")
|
|
|
|
|
|
|
|
|
|
val externalHeartbeatProps = Props(new CrossDcHeartbeatSender()).withDispatcher(UseDispatcher)
|
|
|
|
|
context.actorOf(externalHeartbeatProps, name = "crossDcHeartbeatSender")
|
|
|
|
|
|
2013-09-17 14:20:29 +02:00
|
|
|
// make sure that join process is stopped
|
|
|
|
|
stopSeedNodeProcess()
|
2017-08-21 10:49:56 +02:00
|
|
|
joinSeedNodesDeadline = None
|
2013-05-24 15:38:24 +02:00
|
|
|
context.become(initialized)
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
def initialized: Actor.Receive = ({
|
2017-05-19 07:20:29 -05:00
|
|
|
case msg: GossipEnvelope ⇒ receiveGossip(msg)
|
|
|
|
|
case msg: GossipStatus ⇒ receiveGossipStatus(msg)
|
|
|
|
|
case GossipTick ⇒ gossipTick()
|
|
|
|
|
case GossipSpeedupTick ⇒ gossipSpeedupTick()
|
|
|
|
|
case ReapUnreachableTick ⇒ reapUnreachableMembers()
|
|
|
|
|
case LeaderActionsTick ⇒ leaderActions()
|
|
|
|
|
case PublishStatsTick ⇒ publishInternalStats()
|
|
|
|
|
case InitJoin ⇒
|
|
|
|
|
logInfo("Received InitJoin message from [{}] to [{}]", sender(), selfAddress)
|
|
|
|
|
initJoin()
|
2013-04-11 09:18:12 +02:00
|
|
|
case Join(node, roles) ⇒ joining(node, roles)
|
|
|
|
|
case ClusterUserAction.Down(address) ⇒ downing(address)
|
|
|
|
|
case ClusterUserAction.Leave(address) ⇒ leaving(address)
|
|
|
|
|
case SendGossipTo(address) ⇒ sendGossipTo(address)
|
|
|
|
|
case msg: SubscriptionMessage ⇒ publisher forward msg
|
2013-08-27 15:14:53 +02:00
|
|
|
case QuarantinedEvent(address, uid) ⇒ quarantined(UniqueAddress(address, uid))
|
2013-04-11 09:18:12 +02:00
|
|
|
case ClusterUserAction.JoinTo(address) ⇒
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Trying to join [{}] when already part of a cluster, ignoring", address)
|
2013-06-24 11:46:29 +02:00
|
|
|
case JoinSeedNodes(seedNodes) ⇒
|
2016-06-02 14:06:57 +02:00
|
|
|
logInfo(
|
|
|
|
|
"Trying to join seed nodes [{}] when already part of a cluster, ignoring",
|
2013-06-24 11:46:29 +02:00
|
|
|
seedNodes.mkString(", "))
|
2016-12-01 18:49:38 +01:00
|
|
|
case ExitingConfirmed(address) ⇒ receiveExitingConfirmed(address)
|
|
|
|
|
}: Actor.Receive).orElse(receiveExitingCompleted)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
def receiveExitingCompleted: Actor.Receive = {
|
|
|
|
|
case ExitingCompleted ⇒
|
|
|
|
|
exitingCompleted()
|
|
|
|
|
sender() ! Done // reply to ask
|
2012-08-19 20:15:22 +02:00
|
|
|
}
|
|
|
|
|
|
2012-08-14 13:55:22 +02:00
|
|
|
def receive = uninitialized
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-04-28 22:28:20 +02:00
|
|
|
override def unhandled(message: Any): Unit = message match {
|
2016-12-01 18:49:38 +01:00
|
|
|
case _: Tick ⇒
|
|
|
|
|
case _: GossipEnvelope ⇒
|
|
|
|
|
case _: GossipStatus ⇒
|
|
|
|
|
case _: ExitingConfirmed ⇒
|
|
|
|
|
case other ⇒ super.unhandled(other)
|
2013-04-28 22:28:20 +02:00
|
|
|
}
|
|
|
|
|
|
2015-05-30 16:12:22 +02:00
|
|
|
def initJoin(): Unit = {
|
|
|
|
|
val selfStatus = latestGossip.member(selfUniqueAddress).status
|
2017-07-04 21:58:03 +02:00
|
|
|
if (removeUnreachableWithMemberStatus.contains(selfStatus)) {
|
2015-05-30 16:12:22 +02:00
|
|
|
// prevents a Down and Exiting node from being used for joining
|
2017-05-19 07:20:29 -05:00
|
|
|
logInfo("Sending InitJoinNack message from node [{}] to [{}]", selfAddress, sender())
|
2015-05-30 16:12:22 +02:00
|
|
|
sender() ! InitJoinNack(selfAddress)
|
2017-05-19 07:20:29 -05:00
|
|
|
} else {
|
|
|
|
|
logInfo("Sending InitJoinAck message from node [{}] to [{}]", selfAddress, sender())
|
2015-05-30 16:12:22 +02:00
|
|
|
sender() ! InitJoinAck(selfAddress)
|
2017-05-19 07:20:29 -05:00
|
|
|
}
|
2015-05-30 16:12:22 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2015-05-04 08:35:46 +02:00
|
|
|
def joinSeedNodes(newSeedNodes: immutable.IndexedSeq[Address]): Unit = {
|
|
|
|
|
if (newSeedNodes.nonEmpty) {
|
2013-09-17 14:20:29 +02:00
|
|
|
stopSeedNodeProcess()
|
2015-05-04 08:35:46 +02:00
|
|
|
seedNodes = newSeedNodes // keep them for retry
|
2013-05-16 17:00:04 +02:00
|
|
|
seedNodeProcess =
|
2015-05-04 08:35:46 +02:00
|
|
|
if (newSeedNodes == immutable.IndexedSeq(selfAddress)) {
|
2013-05-16 17:00:04 +02:00
|
|
|
self ! ClusterUserAction.JoinTo(selfAddress)
|
|
|
|
|
None
|
|
|
|
|
} else {
|
2014-04-03 09:53:20 +02:00
|
|
|
// use unique name of this actor, stopSeedNodeProcess doesn't wait for termination
|
|
|
|
|
seedNodeProcessCounter += 1
|
2015-05-04 08:35:46 +02:00
|
|
|
if (newSeedNodes.head == selfAddress) {
|
|
|
|
|
Some(context.actorOf(Props(classOf[FirstSeedNodeProcess], newSeedNodes).
|
2014-04-03 09:53:20 +02:00
|
|
|
withDispatcher(UseDispatcher), name = "firstSeedNodeProcess-" + seedNodeProcessCounter))
|
|
|
|
|
} else {
|
2015-05-04 08:35:46 +02:00
|
|
|
Some(context.actorOf(Props(classOf[JoinSeedNodeProcess], newSeedNodes).
|
2014-04-03 09:53:20 +02:00
|
|
|
withDispatcher(UseDispatcher), name = "joinSeedNodeProcess-" + seedNodeProcessCounter))
|
|
|
|
|
}
|
2013-05-16 17:00:04 +02:00
|
|
|
}
|
|
|
|
|
}
|
2012-09-06 21:48:40 +02:00
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2013-04-11 09:18:12 +02:00
|
|
|
* Try to join this cluster node with the node specified by `address`.
|
|
|
|
|
* It's only allowed to join from an empty state, i.e. when not already a member.
|
|
|
|
|
* A `Join(selfUniqueAddress)` command is sent to the node to join,
|
|
|
|
|
* which will reply with a `Welcome` message.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
|
|
|
|
def join(address: Address): Unit = {
|
2013-01-15 09:35:07 +01:00
|
|
|
if (address.protocol != selfAddress.protocol)
|
2016-06-02 14:06:57 +02: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)
|
2016-06-02 14:06:57 +02: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)
|
2013-04-11 09:18:12 +02:00
|
|
|
else {
|
|
|
|
|
require(latestGossip.members.isEmpty, "Join can only be done from empty state")
|
|
|
|
|
|
2013-02-17 17:35:43 +01:00
|
|
|
// to support manual join when joining to seed nodes is stuck (no seed nodes available)
|
2013-09-17 14:20:29 +02:00
|
|
|
stopSeedNodeProcess()
|
2013-02-17 17:35:43 +01:00
|
|
|
|
2013-04-11 09:18:12 +02:00
|
|
|
if (address == selfAddress) {
|
2013-05-24 15:38:24 +02:00
|
|
|
becomeInitialized()
|
2013-04-11 09:18:12 +02:00
|
|
|
joining(selfUniqueAddress, cluster.selfRoles)
|
|
|
|
|
} else {
|
|
|
|
|
val joinDeadline = RetryUnsuccessfulJoinAfter match {
|
2013-05-16 17:00:04 +02:00
|
|
|
case d: FiniteDuration ⇒ Some(Deadline.now + d)
|
|
|
|
|
case _ ⇒ None
|
2013-04-11 09:18:12 +02:00
|
|
|
}
|
|
|
|
|
context.become(tryingToJoin(address, joinDeadline))
|
|
|
|
|
clusterCore(address) ! Join(selfUniqueAddress, cluster.selfRoles)
|
2013-03-20 10:32:18 +01:00
|
|
|
}
|
2012-08-19 20:15:22 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2013-09-17 14:20:29 +02:00
|
|
|
def stopSeedNodeProcess(): Unit = {
|
|
|
|
|
seedNodeProcess match {
|
|
|
|
|
case Some(s) ⇒
|
|
|
|
|
// manual join, abort current seedNodeProcess
|
|
|
|
|
context stop s
|
|
|
|
|
seedNodeProcess = None
|
|
|
|
|
case None ⇒ // no seedNodeProcess in progress
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* State transition to JOINING - new node joining.
|
2013-04-11 09:18:12 +02:00
|
|
|
* Received `Join` message and replies with `Welcome` message, containing
|
|
|
|
|
* current gossip state, including the new joining member.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2017-07-04 09:09:40 +01:00
|
|
|
def joining(joiningNode: UniqueAddress, roles: Set[String]): Unit = {
|
2015-05-30 16:12:22 +02:00
|
|
|
val selfStatus = latestGossip.member(selfUniqueAddress).status
|
2017-07-04 09:09:40 +01:00
|
|
|
if (joiningNode.address.protocol != selfAddress.protocol)
|
2016-06-02 14:06:57 +02:00
|
|
|
log.warning(
|
|
|
|
|
"Member with wrong protocol tried to join, but was ignored, expected [{}] but was [{}]",
|
2017-07-04 09:09:40 +01:00
|
|
|
selfAddress.protocol, joiningNode.address.protocol)
|
|
|
|
|
else if (joiningNode.address.system != selfAddress.system)
|
2016-06-02 14:06:57 +02:00
|
|
|
log.warning(
|
|
|
|
|
"Member with wrong ActorSystem name tried to join, but was ignored, expected [{}] but was [{}]",
|
2017-07-04 09:09:40 +01:00
|
|
|
selfAddress.system, joiningNode.address.system)
|
2017-07-04 21:58:03 +02:00
|
|
|
else if (removeUnreachableWithMemberStatus.contains(selfStatus))
|
2017-07-04 09:09:40 +01:00
|
|
|
logInfo("Trying to join [{}] to [{}] member, ignoring. Use a member that is Up instead.", joiningNode, selfStatus)
|
2013-02-17 17:35:43 +01:00
|
|
|
else {
|
|
|
|
|
val localMembers = latestGossip.members
|
|
|
|
|
|
2013-04-11 09:18:12 +02:00
|
|
|
// check by address without uid to make sure that node with same host:port is not allowed
|
|
|
|
|
// to join until previous node with that host:port has been removed from the cluster
|
2017-07-04 09:09:40 +01:00
|
|
|
localMembers.find(_.address == joiningNode.address) match {
|
|
|
|
|
case Some(m) if m.uniqueAddress == joiningNode ⇒
|
2015-02-07 14:43:44 +01:00
|
|
|
// node retried join attempt, probably due to lost Welcome message
|
|
|
|
|
logInfo("Existing member [{}] is joining again.", m)
|
2017-07-04 09:09:40 +01:00
|
|
|
if (joiningNode != selfUniqueAddress)
|
2015-02-07 14:43:44 +01:00
|
|
|
sender() ! Welcome(selfUniqueAddress, latestGossip)
|
|
|
|
|
case Some(m) ⇒
|
|
|
|
|
// node restarted, same host:port as existing member, but with different uid
|
|
|
|
|
// safe to down and later remove existing member
|
|
|
|
|
// new node will retry join
|
|
|
|
|
logInfo("New incarnation of existing member [{}] is trying to join. " +
|
|
|
|
|
"Existing will be removed from the cluster and then new member will be allowed to join.", m)
|
2016-08-19 11:56:55 +02:00
|
|
|
if (m.status != Down) {
|
|
|
|
|
// we can confirm it as terminated/unreachable immediately
|
|
|
|
|
val newReachability = latestGossip.overview.reachability.terminated(selfUniqueAddress, m.uniqueAddress)
|
|
|
|
|
val newOverview = latestGossip.overview.copy(reachability = newReachability)
|
|
|
|
|
val newGossip = latestGossip.copy(overview = newOverview)
|
|
|
|
|
updateLatestGossip(newGossip)
|
|
|
|
|
|
2015-02-07 14:43:44 +01:00
|
|
|
downing(m.address)
|
2016-08-19 11:56:55 +02:00
|
|
|
}
|
2015-02-07 14:43:44 +01:00
|
|
|
case None ⇒
|
|
|
|
|
// remove the node from the failure detector
|
2017-07-04 09:09:40 +01:00
|
|
|
failureDetector.remove(joiningNode.address)
|
2017-08-22 15:02:27 +02:00
|
|
|
crossDcFailureDetector.remove(joiningNode.address)
|
2015-02-07 14:43:44 +01:00
|
|
|
|
|
|
|
|
// add joining node as Joining
|
|
|
|
|
// add self in case someone else joins before self has joined (Set discards duplicates)
|
2017-07-04 09:09:40 +01:00
|
|
|
val newMembers = localMembers + Member(joiningNode, roles) + Member(selfUniqueAddress, cluster.selfRoles)
|
2015-02-07 14:43:44 +01:00
|
|
|
val newGossip = latestGossip copy (members = newMembers)
|
|
|
|
|
|
|
|
|
|
updateLatestGossip(newGossip)
|
|
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
logInfo("Node [{}] is JOINING, roles [{}]", joiningNode.address, roles.mkString(", "))
|
|
|
|
|
if (joiningNode == selfUniqueAddress) {
|
2015-10-21 07:53:12 +02:00
|
|
|
if (localMembers.isEmpty)
|
|
|
|
|
leaderActions() // important for deterministic oldest when bootstrapping
|
|
|
|
|
} else
|
2015-02-07 14:43:44 +01:00
|
|
|
sender() ! Welcome(selfUniqueAddress, latestGossip)
|
|
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-11 09:18:12 +02:00
|
|
|
/**
|
|
|
|
|
* Reply from Join request.
|
|
|
|
|
*/
|
|
|
|
|
def welcome(joinWith: Address, from: UniqueAddress, gossip: Gossip): Unit = {
|
|
|
|
|
require(latestGossip.members.isEmpty, "Join can only be done from empty state")
|
|
|
|
|
if (joinWith != from.address)
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Ignoring welcome from [{}] when trying to join with [{}]", from.address, joinWith)
|
2013-04-11 09:18:12 +02:00
|
|
|
else {
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState = membershipState.copy(latestGossip = gossip).seen()
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Welcome from [{}]", from.address)
|
2015-02-11 22:16:27 +01:00
|
|
|
assertLatestGossip()
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2013-04-11 09:18:12 +02:00
|
|
|
if (from != selfUniqueAddress)
|
2014-01-16 15:16:35 +01:00
|
|
|
gossipTo(from, sender())
|
2013-05-24 15:38:24 +02:00
|
|
|
becomeInitialized()
|
2013-04-11 09:18:12 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* State transition to LEAVING.
|
2013-04-11 09:18:12 +02:00
|
|
|
* The node will eventually be removed by the leader, after hand-off in EXITING, and only after
|
|
|
|
|
* removal a new node with same address can join the cluster through the normal joining procedure.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
|
|
|
|
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)
|
2018-01-04 13:13:25 +05:30
|
|
|
if (latestGossip.members.exists(m ⇒ m.address == address && (m.status == Joining || m.status == WeaklyUp || 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
|
|
|
|
2013-05-09 09:49:59 +02:00
|
|
|
updateLatestGossip(newGossip)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Marked address [{}] as [{}]", address, Leaving)
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2016-11-17 19:07:22 +01:00
|
|
|
// immediate gossip to speed up the leaving process
|
|
|
|
|
gossip()
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
def exitingCompleted() = {
|
|
|
|
|
logInfo("Exiting completed")
|
|
|
|
|
// ExitingCompleted sent via CoordinatedShutdown to continue the leaving process.
|
|
|
|
|
exitingTasksInProgress = false
|
|
|
|
|
// mark as seen
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState = membershipState.seen()
|
2016-12-01 18:49:38 +01:00
|
|
|
assertLatestGossip()
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2016-12-01 18:49:38 +01:00
|
|
|
|
|
|
|
|
// Let others know (best effort) before shutdown. Otherwise they will not see
|
|
|
|
|
// convergence of the Exiting state until they have detected this node as
|
|
|
|
|
// unreachable and the required downing has finished. They will still need to detect
|
|
|
|
|
// unreachable, but Exiting unreachable will be removed without downing, i.e.
|
|
|
|
|
// normally the leaving of a leader will be graceful without the need
|
|
|
|
|
// for downing. However, if those final gossip messages never arrive it is
|
|
|
|
|
// alright to require the downing, because that is probably caused by a
|
|
|
|
|
// network failure anyway.
|
|
|
|
|
gossipRandomN(NumberOfGossipsBeforeShutdownWhenLeaderExits)
|
|
|
|
|
|
|
|
|
|
// send ExitingConfirmed to two potential leaders
|
2017-07-04 09:09:40 +01:00
|
|
|
val membersExceptSelf = latestGossip.members.filter(_.uniqueAddress != selfUniqueAddress)
|
|
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState.leaderOf(membersExceptSelf) match {
|
2016-12-01 18:49:38 +01:00
|
|
|
case Some(node1) ⇒
|
|
|
|
|
clusterCore(node1.address) ! ExitingConfirmed(selfUniqueAddress)
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState.leaderOf(membersExceptSelf.filterNot(_.uniqueAddress == node1)) match {
|
2016-12-01 18:49:38 +01:00
|
|
|
case Some(node2) ⇒
|
|
|
|
|
clusterCore(node2.address) ! ExitingConfirmed(selfUniqueAddress)
|
|
|
|
|
case None ⇒ // no more potential leader
|
|
|
|
|
}
|
|
|
|
|
case None ⇒ // no leader
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
shutdown()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receiveExitingConfirmed(node: UniqueAddress): Unit = {
|
|
|
|
|
logInfo("Exiting confirmed [{}]", node.address)
|
|
|
|
|
exitingConfirmed += node
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def cleanupExitingConfirmed(): Unit = {
|
|
|
|
|
// in case the actual removal was performed by another leader node we
|
|
|
|
|
if (exitingConfirmed.nonEmpty)
|
|
|
|
|
exitingConfirmed = exitingConfirmed.filter(n ⇒ latestGossip.members.exists(_.uniqueAddress == n))
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2013-08-27 15:14:53 +02:00
|
|
|
* This method is called when a member sees itself as Exiting or Down.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-05-23 13:36:35 +02:00
|
|
|
def shutdown(): Unit = cluster.shutdown()
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
/**
|
2013-08-27 15:14:53 +02:00
|
|
|
* State transition to DOWN.
|
|
|
|
|
* Its status is set to DOWN. The node is also removed from the `seen` table.
|
2012-07-05 13:55:08 +02:00
|
|
|
*
|
2013-04-11 09:18:12 +02:00
|
|
|
* The node will eventually be removed by the leader, and only after removal a new node with same address can
|
|
|
|
|
* join the cluster through the normal joining procedure.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
|
|
|
|
def downing(address: Address): Unit = {
|
|
|
|
|
val localGossip = latestGossip
|
|
|
|
|
val localMembers = localGossip.members
|
2017-07-04 21:58:03 +02:00
|
|
|
val localReachability = membershipState.dcReachability
|
2012-07-07 20:55:02 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
// check if the node to DOWN is in the `members` set
|
2015-05-30 16:12:22 +02:00
|
|
|
localMembers.find(_.address == address) match {
|
2017-07-04 09:09:40 +01:00
|
|
|
case Some(m) if m.status != Down ⇒
|
2013-08-27 15:14:53 +02:00
|
|
|
if (localReachability.isReachable(m.uniqueAddress))
|
|
|
|
|
logInfo("Marking node [{}] as [{}]", m.address, Down)
|
|
|
|
|
else
|
|
|
|
|
logInfo("Marking unreachable node [{}] as [{}]", m.address, Down)
|
|
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
val newGossip = localGossip.markAsDown(m)
|
2013-08-27 15:14:53 +02:00
|
|
|
updateLatestGossip(newGossip)
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2015-05-30 16:12:22 +02:00
|
|
|
case Some(_) ⇒ // already down
|
2013-08-27 15:14:53 +02:00
|
|
|
case None ⇒
|
2016-08-19 11:56:55 +02:00
|
|
|
logInfo("Ignoring down of unknown node [{}]", address)
|
2013-08-27 15:14:53 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
def quarantined(node: UniqueAddress): Unit = {
|
|
|
|
|
val localGossip = latestGossip
|
|
|
|
|
if (localGossip.hasMember(node)) {
|
|
|
|
|
val newReachability = latestGossip.overview.reachability.terminated(selfUniqueAddress, node)
|
|
|
|
|
val newOverview = localGossip.overview copy (reachability = newReachability)
|
|
|
|
|
val newGossip = localGossip copy (overview = newOverview)
|
|
|
|
|
updateLatestGossip(newGossip)
|
2016-06-02 14:06:57 +02:00
|
|
|
log.warning(
|
|
|
|
|
"Cluster Node [{}] - Marking node as TERMINATED [{}], due to quarantine. Node roles [{}]",
|
2016-05-24 14:35:50 +02:00
|
|
|
selfAddress, node.address, selfRoles.mkString(","))
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2013-08-27 15:14:53 +02:00
|
|
|
downing(node.address)
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:28:20 +02:00
|
|
|
def receiveGossipStatus(status: GossipStatus): Unit = {
|
|
|
|
|
val from = status.from
|
2017-10-31 14:05:51 +01:00
|
|
|
if (!latestGossip.hasMember(from))
|
|
|
|
|
logInfo("Ignoring received gossip status from unknown [{}]", from)
|
|
|
|
|
else if (!latestGossip.isReachable(selfUniqueAddress, from))
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Ignoring received gossip status from unreachable [{}] ", from)
|
2013-04-28 22:28:20 +02:00
|
|
|
else {
|
2013-06-13 15:43:37 -04:00
|
|
|
(status.version compareTo latestGossip.version) match {
|
|
|
|
|
case VectorClock.Same ⇒ // same version
|
2014-01-16 15:16:35 +01:00
|
|
|
case VectorClock.After ⇒ gossipStatusTo(from, sender()) // remote is newer
|
|
|
|
|
case _ ⇒ gossipTo(from, sender()) // conflicting or local is newer
|
2013-04-28 22:28:20 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-06-13 15:43:37 -04:00
|
|
|
/**
|
|
|
|
|
* The types of gossip actions that receive gossip has performed.
|
|
|
|
|
*/
|
|
|
|
|
sealed trait ReceiveGossipType
|
|
|
|
|
case object Ignored extends ReceiveGossipType
|
|
|
|
|
case object Older extends ReceiveGossipType
|
|
|
|
|
case object Newer extends ReceiveGossipType
|
|
|
|
|
case object Same extends ReceiveGossipType
|
|
|
|
|
case object Merge extends ReceiveGossipType
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* Receive new gossip.
|
|
|
|
|
*/
|
2013-06-13 15:43:37 -04:00
|
|
|
def receiveGossip(envelope: GossipEnvelope): ReceiveGossipType = {
|
2017-07-04 09:09:40 +01:00
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
val from = envelope.from
|
|
|
|
|
val remoteGossip = envelope.gossip
|
|
|
|
|
val localGossip = latestGossip
|
|
|
|
|
|
2013-10-17 10:35:37 +02:00
|
|
|
if (remoteGossip eq Gossip.empty) {
|
|
|
|
|
log.debug("Cluster Node [{}] - Ignoring received gossip from [{}] to protect against overload", selfAddress, from)
|
|
|
|
|
Ignored
|
|
|
|
|
} else if (envelope.to != selfUniqueAddress) {
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Ignoring received gossip intended for someone else, from [{}] to [{}]", from.address, envelope.to)
|
2013-06-13 15:43:37 -04:00
|
|
|
Ignored
|
2017-10-31 14:05:51 +01:00
|
|
|
} else if (!localGossip.hasMember(from)) {
|
|
|
|
|
logInfo("Ignoring received gossip from unknown [{}]", from)
|
|
|
|
|
Ignored
|
2017-07-04 09:09:40 +01:00
|
|
|
} else if (!localGossip.isReachable(selfUniqueAddress, from)) {
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Ignoring received gossip from unreachable [{}] ", from)
|
2013-06-13 15:43:37 -04:00
|
|
|
Ignored
|
|
|
|
|
} else if (remoteGossip.members.forall(_.uniqueAddress != selfUniqueAddress)) {
|
2013-05-23 13:36:35 +02:00
|
|
|
logInfo("Ignoring received gossip that does not contain myself, from [{}]", from)
|
2013-06-13 15:43:37 -04:00
|
|
|
Ignored
|
|
|
|
|
} else {
|
|
|
|
|
val comparison = remoteGossip.version compareTo localGossip.version
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-06-13 15:43:37 -04:00
|
|
|
val (winningGossip, talkback, gossipType) = comparison match {
|
|
|
|
|
case VectorClock.Same ⇒
|
2013-03-07 13:20:50 +01:00
|
|
|
// same version
|
2016-12-01 18:49:38 +01:00
|
|
|
val talkback = !exitingTasksInProgress && !remoteGossip.seenByNode(selfUniqueAddress)
|
|
|
|
|
(remoteGossip mergeSeen localGossip, talkback, Same)
|
2013-06-13 15:43:37 -04:00
|
|
|
case VectorClock.Before ⇒
|
2013-03-07 13:20:50 +01:00
|
|
|
// local is newer
|
2013-06-13 15:43:37 -04:00
|
|
|
(localGossip, true, Older)
|
|
|
|
|
case VectorClock.After ⇒
|
2013-03-07 13:20:50 +01:00
|
|
|
// remote is newer
|
2016-12-01 18:49:38 +01:00
|
|
|
val talkback = !exitingTasksInProgress && !remoteGossip.seenByNode(selfUniqueAddress)
|
|
|
|
|
(remoteGossip, talkback, Newer)
|
2013-09-02 13:50:46 +02:00
|
|
|
case _ ⇒
|
|
|
|
|
// conflicting versions, merge
|
2015-02-11 22:16:27 +01:00
|
|
|
// We can see that a removal was done when it is not in one of the gossips has status
|
|
|
|
|
// Down or Exiting in the other gossip.
|
|
|
|
|
// Perform the same pruning (clear of VectorClock) as the leader did when removing a member.
|
|
|
|
|
// Removal of member itself is handled in merge (pickHighestPriority)
|
|
|
|
|
val prunedLocalGossip = localGossip.members.foldLeft(localGossip) { (g, m) ⇒
|
2017-07-04 21:58:03 +02:00
|
|
|
if (removeUnreachableWithMemberStatus(m.status) && !remoteGossip.members.contains(m)) {
|
2015-02-11 22:16:27 +01:00
|
|
|
log.debug("Cluster Node [{}] - Pruned conflicting local gossip: {}", selfAddress, m)
|
2017-07-10 13:01:06 +01:00
|
|
|
g.prune(VectorClock.Node(Gossip.vclockName(m.uniqueAddress)))
|
2015-02-11 22:16:27 +01:00
|
|
|
} else
|
|
|
|
|
g
|
|
|
|
|
}
|
|
|
|
|
val prunedRemoteGossip = remoteGossip.members.foldLeft(remoteGossip) { (g, m) ⇒
|
2017-07-04 21:58:03 +02:00
|
|
|
if (removeUnreachableWithMemberStatus(m.status) && !localGossip.members.contains(m)) {
|
2015-02-11 22:16:27 +01:00
|
|
|
log.debug("Cluster Node [{}] - Pruned conflicting remote gossip: {}", selfAddress, m)
|
2017-07-10 13:01:06 +01:00
|
|
|
g.prune(VectorClock.Node(Gossip.vclockName(m.uniqueAddress)))
|
2015-02-11 22:16:27 +01:00
|
|
|
} else
|
|
|
|
|
g
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
(prunedRemoteGossip merge prunedLocalGossip, true, Merge)
|
2013-03-07 13:20:50 +01:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
// Don't mark gossip state as seen while exiting is in progress, e.g.
|
|
|
|
|
// shutting down singleton actors. This delays removal of the member until
|
|
|
|
|
// the exiting tasks have been completed.
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState = membershipState.copy(latestGossip =
|
2017-07-04 09:09:40 +01:00
|
|
|
if (exitingTasksInProgress) winningGossip
|
2017-07-04 21:58:03 +02:00
|
|
|
else winningGossip seen selfUniqueAddress)
|
2015-02-11 22:16:27 +01:00
|
|
|
assertLatestGossip()
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-09-04 13:21:34 +02:00
|
|
|
// for all new nodes we remove them from the failure detector
|
2013-03-08 13:26:50 +01:00
|
|
|
latestGossip.members foreach {
|
2017-08-22 15:02:27 +02:00
|
|
|
node ⇒
|
2017-09-04 13:21:34 +02:00
|
|
|
if (!localGossip.members(node)) {
|
2017-08-22 15:02:27 +02:00
|
|
|
failureDetector.remove(node.address)
|
|
|
|
|
crossDcFailureDetector.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
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
if (comparison == VectorClock.Concurrent && cluster.settings.Debug.VerboseGossipLogging) {
|
2013-03-07 13:20:50 +01:00
|
|
|
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-05-27 16:34:31 +02:00
|
|
|
if (statsEnabled) {
|
2013-06-13 15:43:37 -04:00
|
|
|
gossipStats = gossipType match {
|
2015-01-30 16:34:27 +01:00
|
|
|
case Merge ⇒ gossipStats.incrementMergeCount
|
|
|
|
|
case Same ⇒ gossipStats.incrementSameCount
|
|
|
|
|
case Newer ⇒ gossipStats.incrementNewerCount
|
|
|
|
|
case Older ⇒ gossipStats.incrementOlderCount
|
|
|
|
|
case Ignored ⇒ gossipStats // included in receivedGossipCount
|
2013-05-27 16:34:31 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
val selfStatus = latestGossip.member(selfUniqueAddress).status
|
2016-12-01 18:49:38 +01:00
|
|
|
if (selfStatus == Exiting && !exitingTasksInProgress) {
|
|
|
|
|
// ExitingCompleted will be received via CoordinatedShutdown to continue
|
|
|
|
|
// the leaving process. Meanwhile the gossip state is not marked as seen.
|
|
|
|
|
exitingTasksInProgress = true
|
|
|
|
|
logInfo("Exiting, starting coordinated shutdown")
|
|
|
|
|
selfExiting.trySuccess(Done)
|
2017-12-04 12:22:59 +01:00
|
|
|
coordShutdown.run(CoordinatedShutdown.ClusterLeavingReason)
|
2016-12-01 18:49:38 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (talkback) {
|
2014-01-16 15:16:35 +01:00
|
|
|
// send back gossip to sender() when sender() had different view, i.e. merge, or sender() had
|
|
|
|
|
// older or sender() had newer
|
|
|
|
|
gossipTo(from, sender())
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
2013-06-13 15:43:37 -04:00
|
|
|
gossipType
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-14 14:50:57 +02:00
|
|
|
def gossipTick(): Unit = {
|
|
|
|
|
gossip()
|
|
|
|
|
if (isGossipSpeedupNeeded) {
|
|
|
|
|
scheduler.scheduleOnce(GossipInterval / 3, self, GossipSpeedupTick)
|
|
|
|
|
scheduler.scheduleOnce(GossipInterval * 2 / 3, self, GossipSpeedupTick)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def gossipSpeedupTick(): Unit =
|
|
|
|
|
if (isGossipSpeedupNeeded) gossip()
|
|
|
|
|
|
2017-11-02 18:26:16 +01:00
|
|
|
def isGossipSpeedupNeeded: Boolean = {
|
|
|
|
|
if (latestGossip.isMultiDc)
|
|
|
|
|
latestGossip.overview.seen.count(membershipState.isInSameDc) < latestGossip.members.count(_.dataCenter == cluster.selfDataCenter) / 2
|
|
|
|
|
else
|
|
|
|
|
(latestGossip.overview.seen.size < latestGossip.members.size / 2)
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2016-11-17 19:07:22 +01:00
|
|
|
/**
|
|
|
|
|
* Sends full gossip to `n` other random members.
|
|
|
|
|
*/
|
|
|
|
|
def gossipRandomN(n: Int): Unit = {
|
|
|
|
|
if (!isSingletonCluster && n > 0) {
|
2017-07-07 13:19:10 +01:00
|
|
|
gossipTargetSelector.randomNodesForFullGossip(membershipState, n).foreach(gossipTo)
|
2016-11-17 19:07:22 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* Initiates a new round of gossip.
|
|
|
|
|
*/
|
2017-07-07 13:19:10 +01:00
|
|
|
def gossip(): Unit =
|
2013-08-27 15:14:53 +02:00
|
|
|
if (!isSingletonCluster) {
|
2017-07-07 13:19:10 +01:00
|
|
|
gossipTargetSelector.gossipTarget(membershipState) match {
|
|
|
|
|
case Some(peer) ⇒
|
|
|
|
|
if (!membershipState.isInSameDc(peer) || latestGossip.seenByNode(peer))
|
|
|
|
|
// avoid transferring the full state if possible
|
|
|
|
|
gossipStatusTo(peer)
|
|
|
|
|
else
|
|
|
|
|
gossipTo(peer)
|
|
|
|
|
case None ⇒ // nothing to see here
|
2017-11-02 08:17:24 +00:00
|
|
|
if (cluster.settings.Debug.VerboseGossipLogging)
|
|
|
|
|
log.debug("Cluster Node [{}] dc [{}] will not gossip this round", selfAddress, cluster.settings.SelfDataCenter)
|
|
|
|
|
|
2013-10-18 08:25:52 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2013-09-11 16:09:51 +02:00
|
|
|
* Runs periodic leader actions, such as member status transitions, assigning partitions etc.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2015-05-30 16:12:22 +02:00
|
|
|
def leaderActions(): Unit = {
|
2017-07-04 21:58:03 +02:00
|
|
|
if (membershipState.isLeader(selfUniqueAddress)) {
|
2017-07-04 17:11:21 +02:00
|
|
|
// only run the leader actions if we are the LEADER of the data center
|
2015-01-30 14:30:16 +01:00
|
|
|
val firstNotice = 20
|
|
|
|
|
val periodicNotice = 60
|
2017-07-04 21:58:03 +02:00
|
|
|
if (membershipState.convergence(exitingConfirmed)) {
|
2015-01-30 14:30:16 +01:00
|
|
|
if (leaderActionCounter >= firstNotice)
|
|
|
|
|
logInfo("Leader can perform its duties again")
|
|
|
|
|
leaderActionCounter = 0
|
2013-05-09 09:49:59 +02:00
|
|
|
leaderActionsOnConvergence()
|
2015-01-30 14:30:16 +01:00
|
|
|
} else {
|
2017-08-22 12:00:42 +02:00
|
|
|
leaderActionCounter += 1
|
|
|
|
|
if (cluster.settings.AllowWeaklyUpMembers && leaderActionCounter >= 3)
|
2015-08-25 17:20:05 -05:00
|
|
|
moveJoiningToWeaklyUp()
|
|
|
|
|
|
2015-01-30 14:30:16 +01:00
|
|
|
if (leaderActionCounter == firstNotice || leaderActionCounter % periodicNotice == 0)
|
2016-06-02 14:06:57 +02:00
|
|
|
logInfo(
|
|
|
|
|
"Leader can currently not perform its duties, reachability status: [{}], member status: [{}]",
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState.dcReachabilityExcludingDownedObservers,
|
2017-07-04 09:09:40 +01:00
|
|
|
latestGossip.members.collect {
|
2017-07-04 17:11:21 +02:00
|
|
|
case m if m.dataCenter == selfDc ⇒
|
2017-07-04 09:09:40 +01:00
|
|
|
s"${m.address} ${m.status} seen=${latestGossip.seenByNode(m.uniqueAddress)}"
|
|
|
|
|
}.mkString(", "))
|
2015-01-30 14:30:16 +01:00
|
|
|
}
|
2013-05-09 09:49:59 +02:00
|
|
|
}
|
2016-12-01 18:49:38 +01:00
|
|
|
cleanupExitingConfirmed()
|
2015-05-30 16:12:22 +02:00
|
|
|
shutdownSelfWhenDown()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def shutdownSelfWhenDown(): Unit = {
|
|
|
|
|
if (latestGossip.member(selfUniqueAddress).status == Down) {
|
|
|
|
|
// When all reachable have seen the state this member will shutdown itself when it has
|
|
|
|
|
// status Down. The down commands should spread before we shutdown.
|
2017-07-04 21:58:03 +02:00
|
|
|
val unreachable = membershipState.dcReachability.allUnreachableOrTerminated
|
|
|
|
|
val downed = membershipState.dcMembers.collect { case m if m.status == Down ⇒ m.uniqueAddress }
|
2015-05-30 16:12:22 +02:00
|
|
|
if (downed.forall(node ⇒ unreachable(node) || latestGossip.seenByNode(node))) {
|
|
|
|
|
// the reason for not shutting down immediately is to give the gossip a chance to spread
|
|
|
|
|
// the downing information to other downed nodes, so that they can shutdown themselves
|
|
|
|
|
logInfo("Shutting down myself")
|
|
|
|
|
// not crucial to send gossip, but may speedup removal since fallback to failure detection is not needed
|
|
|
|
|
// if other downed know that this node has seen the version
|
2016-11-17 19:07:22 +01:00
|
|
|
gossipRandomN(MaxGossipsBeforeShuttingDownMyself)
|
2015-05-30 16:12:22 +02:00
|
|
|
shutdown()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2015-08-25 17:20:05 -05:00
|
|
|
def isMinNrOfMembersFulfilled: Boolean = {
|
|
|
|
|
latestGossip.members.size >= MinNrOfMembers && MinNrOfMembersOfRole.forall {
|
|
|
|
|
case (role, threshold) ⇒ latestGossip.members.count(_.hasRole(role)) >= threshold
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-09 09:49:59 +02:00
|
|
|
/**
|
|
|
|
|
* Leader actions are as follows:
|
|
|
|
|
* 1. Move JOINING => UP -- When a node joins the cluster
|
2015-02-07 14:43:44 +01:00
|
|
|
* 2. Move LEAVING => EXITING --
|
|
|
|
|
* 3. Non-exiting remain --
|
2013-05-09 09:49:59 +02:00
|
|
|
* 4. Move unreachable EXITING => REMOVED -- When all nodes have seen the EXITING node as unreachable (convergence) -
|
|
|
|
|
* remove the node from the node ring and seen table
|
|
|
|
|
* 5. Move unreachable DOWN/EXITING => REMOVED -- When all nodes have seen that the node is DOWN/EXITING (convergence) -
|
|
|
|
|
* remove the node from the node ring and seen table
|
|
|
|
|
* 7. Updating the vclock version for the changes
|
|
|
|
|
* 8. Updating the `seen` table
|
|
|
|
|
* 9. Update the state with the new gossip
|
|
|
|
|
*/
|
|
|
|
|
def leaderActionsOnConvergence(): Unit = {
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
val removedUnreachable = for {
|
2017-07-04 21:58:03 +02:00
|
|
|
node ← membershipState.dcReachability.allUnreachableOrTerminated
|
2017-07-04 09:09:40 +01:00
|
|
|
m = latestGossip.member(node)
|
2017-07-04 21:58:03 +02:00
|
|
|
if m.dataCenter == selfDc && removeUnreachableWithMemberStatus(m.status)
|
2013-08-27 15:14:53 +02:00
|
|
|
} yield m
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
val removedExitingConfirmed = exitingConfirmed.filter { n ⇒
|
|
|
|
|
val member = latestGossip.member(n)
|
2017-07-04 17:11:21 +02:00
|
|
|
member.dataCenter == selfDc && member.status == Exiting
|
2013-05-09 09:49:59 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
val changedMembers = {
|
|
|
|
|
val enoughMembers: Boolean = isMinNrOfMembersFulfilled
|
|
|
|
|
def isJoiningToUp(m: Member): Boolean = (m.status == Joining || m.status == WeaklyUp) && enoughMembers
|
|
|
|
|
|
|
|
|
|
latestGossip.members collect {
|
|
|
|
|
var upNumber = 0
|
|
|
|
|
|
|
|
|
|
{
|
2017-07-04 17:11:21 +02:00
|
|
|
case m if m.dataCenter == selfDc && isJoiningToUp(m) ⇒
|
2017-07-04 09:09:40 +01:00
|
|
|
// Move JOINING => UP (once all nodes have seen that this node is JOINING, i.e. we have a convergence)
|
|
|
|
|
// and minimum number of nodes have joined the cluster
|
|
|
|
|
if (upNumber == 0) {
|
|
|
|
|
// It is alright to use same upNumber as already used by a removed member, since the upNumber
|
|
|
|
|
// is only used for comparing age of current cluster members (Member.isOlderThan)
|
2017-09-04 15:36:39 +02:00
|
|
|
val youngest = membershipState.youngestMember
|
2017-07-04 09:09:40 +01:00
|
|
|
upNumber = 1 + (if (youngest.upNumber == Int.MaxValue) 0 else youngest.upNumber)
|
|
|
|
|
} else {
|
|
|
|
|
upNumber += 1
|
|
|
|
|
}
|
|
|
|
|
m.copyUp(upNumber)
|
|
|
|
|
|
2017-07-04 17:11:21 +02:00
|
|
|
case m if m.dataCenter == selfDc && m.status == Leaving ⇒
|
2017-07-04 09:09:40 +01:00
|
|
|
// Move LEAVING => EXITING (once we have a convergence on LEAVING)
|
|
|
|
|
m copy (status = Exiting)
|
|
|
|
|
}
|
2015-02-11 22:16:27 +01:00
|
|
|
}
|
2017-07-04 09:09:40 +01:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
val updatedGossip: Gossip =
|
|
|
|
|
if (removedUnreachable.nonEmpty || removedExitingConfirmed.nonEmpty || changedMembers.nonEmpty) {
|
|
|
|
|
|
|
|
|
|
// replace changed members
|
|
|
|
|
val removed = removedUnreachable.map(_.uniqueAddress).union(removedExitingConfirmed)
|
|
|
|
|
val newGossip =
|
|
|
|
|
latestGossip.update(changedMembers).removeAll(removed, System.currentTimeMillis())
|
|
|
|
|
|
|
|
|
|
if (!exitingTasksInProgress && newGossip.member(selfUniqueAddress).status == Exiting) {
|
|
|
|
|
// Leader is moving itself from Leaving to Exiting.
|
|
|
|
|
// ExitingCompleted will be received via CoordinatedShutdown to continue
|
|
|
|
|
// the leaving process. Meanwhile the gossip state is not marked as seen.
|
|
|
|
|
exitingTasksInProgress = true
|
|
|
|
|
logInfo("Exiting (leader), starting coordinated shutdown")
|
|
|
|
|
selfExiting.trySuccess(Done)
|
2017-12-04 12:22:59 +01:00
|
|
|
coordShutdown.run(CoordinatedShutdown.ClusterLeavingReason)
|
2017-07-04 09:09:40 +01:00
|
|
|
}
|
2016-12-01 18:49:38 +01:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
exitingConfirmed = exitingConfirmed.filterNot(removedExitingConfirmed)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
changedMembers foreach { m ⇒
|
|
|
|
|
logInfo("Leader is moving node [{}] to [{}]", m.address, m.status)
|
|
|
|
|
}
|
|
|
|
|
removedUnreachable foreach { m ⇒
|
|
|
|
|
val status = if (m.status == Exiting) "exiting" else "unreachable"
|
|
|
|
|
logInfo("Leader is removing {} node [{}]", status, m.address)
|
|
|
|
|
}
|
|
|
|
|
removedExitingConfirmed.foreach { n ⇒
|
|
|
|
|
logInfo("Leader is removing confirmed Exiting node [{}]", n.address)
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
newGossip
|
|
|
|
|
} else
|
|
|
|
|
latestGossip
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
val pruned = updatedGossip.pruneTombstones(System.currentTimeMillis() - PruneGossipTombstonesAfter.toMillis)
|
|
|
|
|
if (pruned ne latestGossip) {
|
|
|
|
|
updateLatestGossip(pruned)
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2013-05-09 09:49:59 +02:00
|
|
|
}
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2015-08-25 17:20:05 -05:00
|
|
|
def moveJoiningToWeaklyUp(): Unit = {
|
|
|
|
|
val localGossip = latestGossip
|
|
|
|
|
val localMembers = localGossip.members
|
|
|
|
|
|
|
|
|
|
val enoughMembers: Boolean = isMinNrOfMembersFulfilled
|
|
|
|
|
def isJoiningToWeaklyUp(m: Member): Boolean =
|
2017-07-04 17:11:21 +02:00
|
|
|
m.dataCenter == selfDc &&
|
2017-07-04 09:09:40 +01:00
|
|
|
m.status == Joining &&
|
|
|
|
|
enoughMembers &&
|
2017-07-04 21:58:03 +02:00
|
|
|
membershipState.dcReachabilityExcludingDownedObservers.isReachable(m.uniqueAddress)
|
2015-08-25 17:20:05 -05:00
|
|
|
val changedMembers = localMembers.collect {
|
|
|
|
|
case m if isJoiningToWeaklyUp(m) ⇒ m.copy(status = WeaklyUp)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (changedMembers.nonEmpty) {
|
|
|
|
|
// replace changed members
|
2015-10-30 14:59:36 +01:00
|
|
|
val newMembers = changedMembers union localMembers
|
2015-08-25 17:20:05 -05:00
|
|
|
val newGossip = localGossip.copy(members = newMembers)
|
|
|
|
|
updateLatestGossip(newGossip)
|
|
|
|
|
|
|
|
|
|
// log status changes
|
|
|
|
|
changedMembers foreach { m ⇒
|
|
|
|
|
logInfo("Leader is moving node [{}] to [{}]", m.address, m.status)
|
|
|
|
|
}
|
|
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2015-08-25 17:20:05 -05:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2013-08-27 15:14:53 +02:00
|
|
|
* Reaps the unreachable members according to the failure detector's verdict.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
|
|
|
|
def reapUnreachableMembers(): Unit = {
|
2013-08-27 15:14:53 +02:00
|
|
|
if (!isSingletonCluster) {
|
|
|
|
|
// only scrutinize if we are a non-singleton cluster
|
2012-07-05 13:55:08 +02:00
|
|
|
|
|
|
|
|
val localGossip = latestGossip
|
|
|
|
|
val localOverview = localGossip.overview
|
|
|
|
|
val localMembers = localGossip.members
|
|
|
|
|
|
2017-08-22 15:02:27 +02:00
|
|
|
def isAvailable(member: Member): Boolean = {
|
|
|
|
|
if (member.dataCenter == SelfDataCenter) failureDetector.isAvailable(member.address)
|
|
|
|
|
else crossDcFailureDetector.isAvailable(member.address)
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
val newlyDetectedUnreachableMembers = localMembers filterNot { member ⇒
|
2013-08-27 15:14:53 +02:00
|
|
|
member.uniqueAddress == selfUniqueAddress ||
|
|
|
|
|
localOverview.reachability.status(selfUniqueAddress, member.uniqueAddress) == Reachability.Unreachable ||
|
|
|
|
|
localOverview.reachability.status(selfUniqueAddress, member.uniqueAddress) == Reachability.Terminated ||
|
2017-08-22 15:02:27 +02:00
|
|
|
isAvailable(member)
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
val newlyDetectedReachableMembers = localOverview.reachability.allUnreachableFrom(selfUniqueAddress) collect {
|
2017-08-22 15:02:27 +02:00
|
|
|
case node if node != selfUniqueAddress && isAvailable(localGossip.member(node)) ⇒
|
2013-08-27 15:14:53 +02:00
|
|
|
localGossip.member(node)
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
if (newlyDetectedUnreachableMembers.nonEmpty || newlyDetectedReachableMembers.nonEmpty) {
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 09:09:40 +01:00
|
|
|
val newReachability1 = newlyDetectedUnreachableMembers.foldLeft(localOverview.reachability) {
|
2013-08-27 15:14:53 +02:00
|
|
|
(reachability, m) ⇒ reachability.unreachable(selfUniqueAddress, m.uniqueAddress)
|
|
|
|
|
}
|
2017-07-04 09:09:40 +01:00
|
|
|
val newReachability2 = newlyDetectedReachableMembers.foldLeft(newReachability1) {
|
2013-08-27 15:14:53 +02:00
|
|
|
(reachability, m) ⇒ reachability.reachable(selfUniqueAddress, m.uniqueAddress)
|
|
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
if (newReachability2 ne localOverview.reachability) {
|
|
|
|
|
val newOverview = localOverview copy (reachability = newReachability2)
|
|
|
|
|
val newGossip = localGossip copy (overview = newOverview)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
updateLatestGossip(newGossip)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-08-27 15:14:53 +02:00
|
|
|
val (exiting, nonExiting) = newlyDetectedUnreachableMembers.partition(_.status == Exiting)
|
|
|
|
|
if (nonExiting.nonEmpty)
|
2016-05-24 14:35:50 +02:00
|
|
|
log.warning("Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]. Node roles [{}]", selfAddress, nonExiting.mkString(", "), selfRoles.mkString(", "))
|
2013-08-27 15:14:53 +02:00
|
|
|
if (exiting.nonEmpty)
|
2016-06-02 14:06:57 +02:00
|
|
|
logInfo(
|
|
|
|
|
"Marking exiting node(s) as UNREACHABLE [{}]. This is expected and they will be removed.",
|
2013-08-27 15:14:53 +02:00
|
|
|
exiting.mkString(", "))
|
|
|
|
|
if (newlyDetectedReachableMembers.nonEmpty)
|
2016-05-24 14:35:50 +02:00
|
|
|
logInfo("Marking node(s) as REACHABLE [{}]. Node roles [{}]", newlyDetectedReachableMembers.mkString(", "), selfRoles.mkString(","))
|
2013-08-27 15:14:53 +02:00
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
publishMembershipState()
|
2013-08-27 15:14:53 +02:00
|
|
|
}
|
2012-07-05 13:55:08 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def isSingletonCluster: Boolean = latestGossip.isSingletonCluster
|
|
|
|
|
|
2013-04-11 09:18:12 +02:00
|
|
|
// needed for tests
|
|
|
|
|
def sendGossipTo(address: Address): Unit = {
|
|
|
|
|
latestGossip.members.foreach(m ⇒
|
|
|
|
|
if (m.address == address)
|
|
|
|
|
gossipTo(m.uniqueAddress))
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
2013-04-11 09:18:12 +02:00
|
|
|
* Gossips latest gossip to a node.
|
2012-07-05 13:55:08 +02:00
|
|
|
*/
|
2013-04-11 09:18:12 +02:00
|
|
|
def gossipTo(node: UniqueAddress): Unit =
|
2017-07-07 13:19:10 +01:00
|
|
|
if (membershipState.validNodeForGossip(node))
|
2013-04-28 22:28:20 +02:00
|
|
|
clusterCore(node.address) ! GossipEnvelope(selfUniqueAddress, node, latestGossip)
|
|
|
|
|
|
|
|
|
|
def gossipTo(node: UniqueAddress, destination: ActorRef): Unit =
|
2017-07-07 13:19:10 +01:00
|
|
|
if (membershipState.validNodeForGossip(node))
|
2013-04-28 22:28:20 +02:00
|
|
|
destination ! GossipEnvelope(selfUniqueAddress, node, latestGossip)
|
|
|
|
|
|
|
|
|
|
def gossipStatusTo(node: UniqueAddress, destination: ActorRef): Unit =
|
2017-07-07 13:19:10 +01:00
|
|
|
if (membershipState.validNodeForGossip(node))
|
2013-04-28 22:28:20 +02:00
|
|
|
destination ! GossipStatus(selfUniqueAddress, latestGossip.version)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-04-28 22:28:20 +02:00
|
|
|
def gossipStatusTo(node: UniqueAddress): Unit =
|
2017-07-07 13:19:10 +01:00
|
|
|
if (membershipState.validNodeForGossip(node))
|
2013-04-28 22:28:20 +02:00
|
|
|
clusterCore(node.address) ! GossipStatus(selfUniqueAddress, latestGossip.version)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
def updateLatestGossip(gossip: Gossip): Unit = {
|
2013-05-09 09:49:59 +02:00
|
|
|
// Updating the vclock version for the changes
|
2017-07-04 21:58:03 +02:00
|
|
|
val versionedGossip = gossip :+ vclockNode
|
2016-12-01 18:49:38 +01:00
|
|
|
|
|
|
|
|
// Don't mark gossip state as seen while exiting is in progress, e.g.
|
|
|
|
|
// shutting down singleton actors. This delays removal of the member until
|
|
|
|
|
// the exiting tasks have been completed.
|
2017-07-04 21:58:03 +02:00
|
|
|
val newGossip =
|
|
|
|
|
if (exitingTasksInProgress)
|
|
|
|
|
versionedGossip.clearSeen()
|
|
|
|
|
else {
|
|
|
|
|
// Nobody else has seen this gossip but us
|
|
|
|
|
val seenVersionedGossip = versionedGossip onlySeen (selfUniqueAddress)
|
|
|
|
|
// Update the state with the new gossip
|
|
|
|
|
seenVersionedGossip
|
|
|
|
|
}
|
|
|
|
|
membershipState = membershipState.copy(newGossip)
|
2015-02-11 22:16:27 +01:00
|
|
|
assertLatestGossip()
|
2013-05-09 09:49:59 +02:00
|
|
|
}
|
|
|
|
|
|
2015-02-11 22:16:27 +01:00
|
|
|
def assertLatestGossip(): Unit =
|
|
|
|
|
if (Cluster.isAssertInvariantsEnabled && latestGossip.version.versions.size > latestGossip.members.size)
|
|
|
|
|
throw new IllegalStateException(s"Too many vector clock entries in gossip state ${latestGossip}")
|
|
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
def publishMembershipState(): Unit = {
|
2017-07-04 09:09:40 +01:00
|
|
|
if (cluster.settings.Debug.VerboseGossipLogging)
|
2017-07-12 11:47:32 +01:00
|
|
|
log.debug("Cluster Node [{}] dc [{}] - New gossip published [{}]", selfAddress, cluster.settings.SelfDataCenter, membershipState.latestGossip)
|
2017-07-04 09:09:40 +01:00
|
|
|
|
2017-07-04 21:58:03 +02:00
|
|
|
publisher ! PublishChanges(membershipState)
|
2012-08-15 16:47:34 +02:00
|
|
|
if (PublishStatsInterval == Duration.Zero) publishInternalStats()
|
2012-08-14 10:58:30 +02:00
|
|
|
}
|
|
|
|
|
|
2013-06-15 23:17:05 +02:00
|
|
|
def publishInternalStats(): Unit = {
|
|
|
|
|
val vclockStats = VectorClockStats(
|
|
|
|
|
versionSize = latestGossip.version.versions.size,
|
|
|
|
|
seenLatest = latestGossip.members.count(m ⇒ latestGossip.seenByNode(m.uniqueAddress)))
|
|
|
|
|
publisher ! CurrentInternalStats(gossipStats, vclockStats)
|
|
|
|
|
}
|
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._
|
2013-04-11 09:18:12 +02:00
|
|
|
import ClusterUserAction.JoinTo
|
2013-02-17 17:35:43 +01:00
|
|
|
|
|
|
|
|
val cluster = Cluster(context.system)
|
2017-05-19 07:20:29 -05:00
|
|
|
import cluster.InfoLogger._
|
|
|
|
|
|
2013-02-17 17:35:43 +01:00
|
|
|
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
|
2016-12-21 16:15:56 +01:00
|
|
|
if (log.isDebugEnabled)
|
|
|
|
|
log.debug(
|
|
|
|
|
"Couldn't join other seed nodes, will join myself. seed-nodes=[{}]",
|
|
|
|
|
seedNodes.mkString(", "))
|
2013-02-17 17:35:43 +01:00
|
|
|
context.parent ! JoinTo(selfAddress)
|
|
|
|
|
context.stop(self)
|
|
|
|
|
}
|
|
|
|
|
case InitJoinAck(address) ⇒
|
2017-05-19 07:20:29 -05:00
|
|
|
logInfo("Received InitJoinAck message from [{}] to [{}]", sender(), selfAddress)
|
2013-02-17 17:35:43 +01:00
|
|
|
// first InitJoinAck reply, join existing cluster
|
|
|
|
|
context.parent ! JoinTo(address)
|
|
|
|
|
context.stop(self)
|
|
|
|
|
case InitJoinNack(address) ⇒
|
2017-05-19 07:20:29 -05:00
|
|
|
logInfo("Received InitJoinNack message from [{}] to [{}]", sender(), selfAddress)
|
2013-02-17 17:35:43 +01:00
|
|
|
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._
|
2013-04-11 09:18:12 +02:00
|
|
|
import ClusterUserAction.JoinTo
|
2012-08-14 17:26:33 +02:00
|
|
|
|
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
|
|
|
|
2016-12-21 16:15:56 +01:00
|
|
|
var attempt = 0
|
|
|
|
|
|
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)
|
2016-12-21 16:15:56 +01:00
|
|
|
attempt += 1
|
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 ⇒
|
2016-12-21 16:15:56 +01:00
|
|
|
if (attempt >= 2)
|
|
|
|
|
log.warning(
|
2017-02-28 11:56:40 -08:00
|
|
|
"Couldn't join seed nodes after [{}] attempts, will try again. seed-nodes=[{}]",
|
2016-12-21 16:15:56 +01:00
|
|
|
attempt, seedNodes.filterNot(_ == selfAddress).mkString(", "))
|
2012-08-15 08:21:34 +02:00
|
|
|
// 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
|
|
|
|
|
*
|
2015-04-17 17:28:37 +08:00
|
|
|
* The supplied callback will be run, once, when current cluster member come up with the same status.
|
2012-12-10 08:46:25 +01:00
|
|
|
*/
|
2015-04-17 17:28:37 +08:00
|
|
|
private[cluster] class OnMemberStatusChangedListener(callback: Runnable, status: MemberStatus) extends Actor with ActorLogging {
|
2012-12-10 08:46:25 +01:00
|
|
|
import ClusterEvent._
|
2015-04-17 17:28:37 +08:00
|
|
|
private val cluster = Cluster(context.system)
|
|
|
|
|
private val to = status match {
|
2016-08-19 11:56:55 +02:00
|
|
|
case Up ⇒ classOf[MemberUp]
|
|
|
|
|
case Removed ⇒ classOf[MemberRemoved]
|
|
|
|
|
case other ⇒ throw new IllegalArgumentException(
|
|
|
|
|
s"Expected Up or Removed in OnMemberStatusChangedListener, got [$other]")
|
2015-04-17 17:28:37 +08:00
|
|
|
}
|
|
|
|
|
|
2012-12-10 08:46:25 +01:00
|
|
|
override def preStart(): Unit =
|
2015-04-17 17:28:37 +08:00
|
|
|
cluster.subscribe(self, to)
|
2015-05-26 09:00:40 +02:00
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
if (status == Removed)
|
|
|
|
|
done()
|
2012-12-10 08:46:25 +01:00
|
|
|
cluster.unsubscribe(self)
|
2015-05-26 09:00:40 +02:00
|
|
|
}
|
2012-12-10 08:46:25 +01:00
|
|
|
|
|
|
|
|
def receive = {
|
|
|
|
|
case state: CurrentClusterState ⇒
|
2015-04-17 17:28:37 +08:00
|
|
|
if (state.members.exists(isTriggered))
|
|
|
|
|
done()
|
|
|
|
|
case MemberUp(member) ⇒
|
|
|
|
|
if (isTriggered(member))
|
2012-12-10 08:46:25 +01:00
|
|
|
done()
|
2015-04-17 17:28:37 +08:00
|
|
|
case MemberRemoved(member, _) ⇒
|
|
|
|
|
if (isTriggered(member))
|
2012-12-10 08:46:25 +01:00
|
|
|
done()
|
|
|
|
|
}
|
|
|
|
|
|
2015-04-17 17:28:37 +08:00
|
|
|
private def done(): Unit = {
|
2012-12-10 08:46:25 +01:00
|
|
|
try callback.run() catch {
|
2015-04-17 17:28:37 +08:00
|
|
|
case NonFatal(e) ⇒ log.error(e, "[{}] callback failed with [{}]", s"On${to.getSimpleName}", e.getMessage)
|
2012-12-10 08:46:25 +01:00
|
|
|
} finally {
|
|
|
|
|
context stop self
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-04-17 17:28:37 +08:00
|
|
|
private def isTriggered(m: Member): Boolean =
|
|
|
|
|
m.uniqueAddress == cluster.selfUniqueAddress && m.status == status
|
2012-12-10 08:46:25 +01:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-05 13:55:08 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2013-04-04 17:56:29 +02:00
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
private[cluster] final case class GossipStats(
|
2012-07-05 13:55:08 +02:00
|
|
|
receivedGossipCount: Long = 0L,
|
2016-08-03 14:06:57 +02:00
|
|
|
mergeCount: Long = 0L,
|
|
|
|
|
sameCount: Long = 0L,
|
|
|
|
|
newerCount: Long = 0L,
|
|
|
|
|
olderCount: Long = 0L) {
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
def incrementMergeCount(): GossipStats =
|
|
|
|
|
copy(mergeCount = mergeCount + 1, receivedGossipCount = receivedGossipCount + 1)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
def incrementSameCount(): GossipStats =
|
|
|
|
|
copy(sameCount = sameCount + 1, receivedGossipCount = receivedGossipCount + 1)
|
2012-07-05 13:55:08 +02:00
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
def incrementNewerCount(): GossipStats =
|
|
|
|
|
copy(newerCount = newerCount + 1, receivedGossipCount = receivedGossipCount + 1)
|
2013-03-05 12:49:35 +01:00
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
def incrementOlderCount(): GossipStats =
|
|
|
|
|
copy(olderCount = olderCount + 1, receivedGossipCount = receivedGossipCount + 1)
|
2013-03-05 12:49:35 +01:00
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
def :+(that: GossipStats): GossipStats = {
|
|
|
|
|
GossipStats(
|
2013-03-05 12:49:35 +01:00
|
|
|
this.receivedGossipCount + that.receivedGossipCount,
|
|
|
|
|
this.mergeCount + that.mergeCount,
|
|
|
|
|
this.sameCount + that.sameCount,
|
|
|
|
|
this.newerCount + that.newerCount,
|
|
|
|
|
this.olderCount + that.olderCount)
|
|
|
|
|
}
|
|
|
|
|
|
2013-05-27 16:34:31 +02:00
|
|
|
def :-(that: GossipStats): GossipStats = {
|
|
|
|
|
GossipStats(
|
2013-03-05 12:49:35 +01:00
|
|
|
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
|
|
|
}
|
2013-05-27 16:34:31 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L)
|
2014-03-07 13:20:01 +01:00
|
|
|
private[cluster] final case class VectorClockStats(
|
2013-05-27 16:34:31 +02:00
|
|
|
versionSize: Int = 0,
|
2016-08-03 14:06:57 +02:00
|
|
|
seenLatest: Int = 0)
|
2013-05-27 16:34:31 +02:00
|
|
|
|