add CoordinatedShutdown, #21537
* CoordinatedShutdown that can run tasks for configured phases in order (DAG) * coordinate handover/shutdown of singleton with cluster exiting/shutdown * phase config obj with depends-on list * integrate graceful leaving of sharding in coordinated shutdown * add timeout and recover * add some missing artery ports to tests * leave via CoordinatedShutdown.run * optionally exit-jvm in last phase * run via jvm shutdown hook * send ExitingConfirmed to leader before shutdown of Exiting to not have to wait for failure detector to mark it as unreachable before removing * the unreachable signal is still kept as a safe guard if message is lost or leader dies * PhaseClusterExiting vs MemberExited in ClusterSingletonManager * terminate ActorSystem when cluster shutdown (via Down) * add more predefined and custom phases * reference documentation * migration guide * problem when the leader order was sys2, sys1, sys3, then sys3 could not perform it's duties and move Leving sys1 to Exiting because it was observing sys1 as unreachable * exclude Leaving with exitingConfirmed from convergence condidtion
This commit is contained in:
parent
4a9c753710
commit
84ade6fdc3
69 changed files with 1778 additions and 339 deletions
|
|
@ -17,6 +17,11 @@ import scala.collection.breakOut
|
|||
import akka.remote.QuarantinedEvent
|
||||
import java.util.ArrayList
|
||||
import java.util.Collections
|
||||
import akka.pattern.ask
|
||||
import akka.util.Timeout
|
||||
import akka.Done
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.Promise
|
||||
|
||||
/**
|
||||
* Base trait for all cluster messages. All ClusterMessage's are serializable.
|
||||
|
|
@ -107,6 +112,8 @@ private[cluster] object InternalClusterAction {
|
|||
@SerialVersionUID(1L)
|
||||
final case class InitJoinNack(address: Address) extends ClusterMessage with DeadLetterSuppression
|
||||
|
||||
final case class ExitingConfirmed(node: UniqueAddress) extends ClusterMessage with DeadLetterSuppression
|
||||
|
||||
/**
|
||||
* Marker interface for periodic tick messages
|
||||
*/
|
||||
|
|
@ -139,8 +146,10 @@ private[cluster] object InternalClusterAction {
|
|||
final case class AddOnMemberRemovedListener(callback: Runnable) extends NoSerializationVerificationNeeded
|
||||
|
||||
sealed trait SubscriptionMessage
|
||||
final case class Subscribe(subscriber: ActorRef, initialStateMode: SubscriptionInitialStateMode, to: Set[Class[_]]) extends SubscriptionMessage
|
||||
final case class Unsubscribe(subscriber: ActorRef, to: Option[Class[_]]) extends SubscriptionMessage
|
||||
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
|
||||
/**
|
||||
* @param receiver [[akka.cluster.ClusterEvent.CurrentClusterState]] will be sent to the `receiver`
|
||||
*/
|
||||
|
|
@ -149,6 +158,9 @@ private[cluster] object InternalClusterAction {
|
|||
sealed trait PublishMessage
|
||||
final case class PublishChanges(newGossip: Gossip) extends PublishMessage
|
||||
final case class PublishEvent(event: ClusterDomainEvent) extends PublishMessage
|
||||
|
||||
final case object ExitingCompleted
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -165,6 +177,30 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac
|
|||
// Child actors are therefore created when GetClusterCoreRef is received
|
||||
var coreSupervisor: Option[ActorRef] = None
|
||||
|
||||
val clusterShutdown = Promise[Done]()
|
||||
val coordShutdown = CoordinatedShutdown(context.system)
|
||||
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterLeave, "leave") {
|
||||
val sys = context.system
|
||||
() ⇒
|
||||
if (Cluster(sys).isTerminated)
|
||||
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) {
|
||||
// run the last phases e.g. if node was downed (not leaving)
|
||||
coordShutdown.run(Some(CoordinatedShutdown.PhaseClusterShutdown))
|
||||
}
|
||||
}
|
||||
|
||||
def createChildren(): Unit = {
|
||||
coreSupervisor = Some(context.actorOf(Props[ClusterCoreSupervisor].
|
||||
withDispatcher(context.props.dispatcher), name = "core"))
|
||||
|
|
@ -188,6 +224,10 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac
|
|||
context.actorOf(Props(classOf[ClusterMetricsCollector], publisher).
|
||||
withDispatcher(context.props.dispatcher), name = "metrics")
|
||||
}
|
||||
case CoordinatedShutdownLeave.LeaveReq ⇒
|
||||
val ref = context.actorOf(CoordinatedShutdownLeave.props().withDispatcher(context.props.dispatcher))
|
||||
// forward the ask request
|
||||
ref.forward(CoordinatedShutdownLeave.LeaveReq)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -267,6 +307,24 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
var seedNodeProcessCounter = 0 // for unique names
|
||||
var leaderActionCounter = 0
|
||||
|
||||
var exitingTasksInProgress = false
|
||||
val selfExiting = Promise[Done]()
|
||||
val coordShutdown = CoordinatedShutdown(context.system)
|
||||
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterExiting, "wait-exiting") { () ⇒
|
||||
selfExiting.future
|
||||
}
|
||||
coordShutdown.addTask(CoordinatedShutdown.PhaseClusterExitingDone, "exiting-completed") {
|
||||
val sys = context.system
|
||||
() ⇒
|
||||
if (Cluster(sys).isTerminated)
|
||||
Future.successful(Done)
|
||||
else {
|
||||
implicit val timeout = Timeout(coordShutdown.timeout(CoordinatedShutdown.PhaseClusterExitingDone))
|
||||
self.ask(ExitingCompleted).mapTo[Done]
|
||||
}
|
||||
}
|
||||
var exitingConfirmed = Set.empty[UniqueAddress]
|
||||
|
||||
/**
|
||||
* Looks up and returns the remote cluster command connection for the specific address.
|
||||
*/
|
||||
|
|
@ -320,16 +378,17 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
failureDetectorReaperTask.cancel()
|
||||
leaderActionsTask.cancel()
|
||||
publishStatsTask foreach { _.cancel() }
|
||||
selfExiting.trySuccess(Done)
|
||||
}
|
||||
|
||||
def uninitialized: Actor.Receive = {
|
||||
def uninitialized: Actor.Receive = ({
|
||||
case InitJoin ⇒ sender() ! InitJoinNack(selfAddress)
|
||||
case ClusterUserAction.JoinTo(address) ⇒ join(address)
|
||||
case JoinSeedNodes(newSeedNodes) ⇒ joinSeedNodes(newSeedNodes)
|
||||
case msg: SubscriptionMessage ⇒ publisher forward msg
|
||||
}
|
||||
}: Actor.Receive).orElse(receiveExitingCompleted)
|
||||
|
||||
def tryingToJoin(joinWith: Address, deadline: Option[Deadline]): Actor.Receive = {
|
||||
def tryingToJoin(joinWith: Address, deadline: Option[Deadline]): Actor.Receive = ({
|
||||
case Welcome(from, gossip) ⇒ welcome(joinWith, from, gossip)
|
||||
case InitJoin ⇒ sender() ! InitJoinNack(selfAddress)
|
||||
case ClusterUserAction.JoinTo(address) ⇒
|
||||
|
|
@ -346,7 +405,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
if (seedNodes.nonEmpty) joinSeedNodes(seedNodes)
|
||||
else join(joinWith)
|
||||
}
|
||||
}
|
||||
}: Actor.Receive).orElse(receiveExitingCompleted)
|
||||
|
||||
def becomeUninitialized(): Unit = {
|
||||
// make sure that join process is stopped
|
||||
|
|
@ -364,7 +423,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
context.become(initialized)
|
||||
}
|
||||
|
||||
def initialized: Actor.Receive = {
|
||||
def initialized: Actor.Receive = ({
|
||||
case msg: GossipEnvelope ⇒ receiveGossip(msg)
|
||||
case msg: GossipStatus ⇒ receiveGossipStatus(msg)
|
||||
case GossipTick ⇒ gossipTick()
|
||||
|
|
@ -385,19 +444,23 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
logInfo(
|
||||
"Trying to join seed nodes [{}] when already part of a cluster, ignoring",
|
||||
seedNodes.mkString(", "))
|
||||
}
|
||||
case ExitingConfirmed(address) ⇒ receiveExitingConfirmed(address)
|
||||
}: Actor.Receive).orElse(receiveExitingCompleted)
|
||||
|
||||
def removed: Actor.Receive = {
|
||||
case msg: SubscriptionMessage ⇒ publisher forward msg
|
||||
def receiveExitingCompleted: Actor.Receive = {
|
||||
case ExitingCompleted ⇒
|
||||
exitingCompleted()
|
||||
sender() ! Done // reply to ask
|
||||
}
|
||||
|
||||
def receive = uninitialized
|
||||
|
||||
override def unhandled(message: Any): Unit = message match {
|
||||
case _: Tick ⇒
|
||||
case _: GossipEnvelope ⇒
|
||||
case _: GossipStatus ⇒
|
||||
case other ⇒ super.unhandled(other)
|
||||
case _: Tick ⇒
|
||||
case _: GossipEnvelope ⇒
|
||||
case _: GossipStatus ⇒
|
||||
case _: ExitingConfirmed ⇒
|
||||
case other ⇒ super.unhandled(other)
|
||||
}
|
||||
|
||||
def initJoin(): Unit = {
|
||||
|
|
@ -580,6 +643,52 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
}
|
||||
}
|
||||
|
||||
def exitingCompleted() = {
|
||||
logInfo("Exiting completed")
|
||||
// ExitingCompleted sent via CoordinatedShutdown to continue the leaving process.
|
||||
exitingTasksInProgress = false
|
||||
// mark as seen
|
||||
latestGossip = latestGossip seen selfUniqueAddress
|
||||
assertLatestGossip()
|
||||
publish(latestGossip)
|
||||
|
||||
// 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
|
||||
val membersWithoutSelf = latestGossip.members.filterNot(_.uniqueAddress == selfUniqueAddress)
|
||||
latestGossip.leaderOf(membersWithoutSelf, selfUniqueAddress) match {
|
||||
case Some(node1) ⇒
|
||||
clusterCore(node1.address) ! ExitingConfirmed(selfUniqueAddress)
|
||||
latestGossip.leaderOf(membersWithoutSelf.filterNot(_.uniqueAddress == node1), selfUniqueAddress) match {
|
||||
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))
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is called when a member sees itself as Exiting or Down.
|
||||
*/
|
||||
|
|
@ -694,13 +803,15 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
val (winningGossip, talkback, gossipType) = comparison match {
|
||||
case VectorClock.Same ⇒
|
||||
// same version
|
||||
(remoteGossip mergeSeen localGossip, !remoteGossip.seenByNode(selfUniqueAddress), Same)
|
||||
val talkback = !exitingTasksInProgress && !remoteGossip.seenByNode(selfUniqueAddress)
|
||||
(remoteGossip mergeSeen localGossip, talkback, Same)
|
||||
case VectorClock.Before ⇒
|
||||
// local is newer
|
||||
(localGossip, true, Older)
|
||||
case VectorClock.After ⇒
|
||||
// remote is newer
|
||||
(remoteGossip, !remoteGossip.seenByNode(selfUniqueAddress), Newer)
|
||||
val talkback = !exitingTasksInProgress && !remoteGossip.seenByNode(selfUniqueAddress)
|
||||
(remoteGossip, talkback, Newer)
|
||||
case _ ⇒
|
||||
// conflicting versions, merge
|
||||
// We can see that a removal was done when it is not in one of the gossips has status
|
||||
|
|
@ -725,7 +836,13 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
(prunedRemoteGossip merge prunedLocalGossip, true, Merge)
|
||||
}
|
||||
|
||||
latestGossip = winningGossip seen selfUniqueAddress
|
||||
// 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.
|
||||
if (exitingTasksInProgress)
|
||||
latestGossip = winningGossip
|
||||
else
|
||||
latestGossip = winningGossip seen selfUniqueAddress
|
||||
assertLatestGossip()
|
||||
|
||||
// for all new joining nodes we remove them from the failure detector
|
||||
|
|
@ -754,9 +871,16 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
publish(latestGossip)
|
||||
|
||||
val selfStatus = latestGossip.member(selfUniqueAddress).status
|
||||
if (selfStatus == Exiting)
|
||||
shutdown()
|
||||
else if (talkback) {
|
||||
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)
|
||||
coordShutdown.run()
|
||||
}
|
||||
|
||||
if (talkback) {
|
||||
// send back gossip to sender() when sender() had different view, i.e. merge, or sender() had
|
||||
// older or sender() had newer
|
||||
gossipTo(from, sender())
|
||||
|
|
@ -875,7 +999,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
// only run the leader actions if we are the LEADER
|
||||
val firstNotice = 20
|
||||
val periodicNotice = 60
|
||||
if (latestGossip.convergence(selfUniqueAddress)) {
|
||||
if (latestGossip.convergence(selfUniqueAddress, exitingConfirmed)) {
|
||||
if (leaderActionCounter >= firstNotice)
|
||||
logInfo("Leader can perform its duties again")
|
||||
leaderActionCounter = 0
|
||||
|
|
@ -893,6 +1017,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
s"${m.address} ${m.status} seen=${latestGossip.seenByNode(m.uniqueAddress)}").mkString(", "))
|
||||
}
|
||||
}
|
||||
cleanupExitingConfirmed()
|
||||
shutdownSelfWhenDown()
|
||||
}
|
||||
|
||||
|
|
@ -948,6 +1073,8 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
if Gossip.removeUnreachableWithMemberStatus(m.status)
|
||||
} yield m
|
||||
|
||||
val removedExitingConfirmed = exitingConfirmed.filter(n ⇒ localGossip.member(n).status == Exiting)
|
||||
|
||||
val changedMembers = localMembers collect {
|
||||
var upNumber = 0
|
||||
|
||||
|
|
@ -971,14 +1098,15 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
}
|
||||
}
|
||||
|
||||
if (removedUnreachable.nonEmpty || changedMembers.nonEmpty) {
|
||||
if (removedUnreachable.nonEmpty || removedExitingConfirmed.nonEmpty || changedMembers.nonEmpty) {
|
||||
// handle changes
|
||||
|
||||
// replace changed members
|
||||
val newMembers = changedMembers union localMembers diff removedUnreachable
|
||||
val newMembers = changedMembers.union(localMembers).diff(removedUnreachable)
|
||||
.filterNot(m ⇒ removedExitingConfirmed(m.uniqueAddress))
|
||||
|
||||
// removing REMOVED nodes from the `seen` table
|
||||
val removed = removedUnreachable.map(_.uniqueAddress)
|
||||
val removed = removedUnreachable.map(_.uniqueAddress).union(removedExitingConfirmed)
|
||||
val newSeen = localSeen diff removed
|
||||
// removing REMOVED nodes from the `reachability` table
|
||||
val newReachability = localOverview.reachability.remove(removed)
|
||||
|
|
@ -992,7 +1120,18 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
}
|
||||
val newGossip = localGossip copy (members = newMembers, overview = newOverview, version = newVersion)
|
||||
|
||||
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)
|
||||
coordShutdown.run()
|
||||
}
|
||||
|
||||
updateLatestGossip(newGossip)
|
||||
exitingConfirmed = exitingConfirmed.filterNot(removedExitingConfirmed)
|
||||
|
||||
// log status changes
|
||||
changedMembers foreach { m ⇒
|
||||
|
|
@ -1004,23 +1143,11 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
val status = if (m.status == Exiting) "exiting" else "unreachable"
|
||||
logInfo("Leader is removing {} node [{}]", status, m.address)
|
||||
}
|
||||
|
||||
publish(latestGossip)
|
||||
|
||||
if (latestGossip.member(selfUniqueAddress).status == Exiting) {
|
||||
// Leader is moving itself from Leaving to Exiting. Let others know (best effort)
|
||||
// before shutdown. Otherwise they will not see the Exiting state change
|
||||
// and there will not be convergence 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)
|
||||
shutdown()
|
||||
removedExitingConfirmed.foreach { n ⇒
|
||||
logInfo("Leader is removing confirmed Exiting node [{}]", n.address)
|
||||
}
|
||||
|
||||
publish(latestGossip)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1144,10 +1271,18 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with
|
|||
def updateLatestGossip(newGossip: Gossip): Unit = {
|
||||
// Updating the vclock version for the changes
|
||||
val versionedGossip = newGossip :+ vclockNode
|
||||
// Nobody else have seen this gossip but us
|
||||
val seenVersionedGossip = versionedGossip onlySeen (selfUniqueAddress)
|
||||
// Update the state with the new gossip
|
||||
latestGossip = seenVersionedGossip
|
||||
|
||||
// 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.
|
||||
if (exitingTasksInProgress)
|
||||
latestGossip = versionedGossip.clearSeen()
|
||||
else {
|
||||
// Nobody else has seen this gossip but us
|
||||
val seenVersionedGossip = versionedGossip onlySeen (selfUniqueAddress)
|
||||
// Update the state with the new gossip
|
||||
latestGossip = seenVersionedGossip
|
||||
}
|
||||
assertLatestGossip()
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue