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:
Patrik Nordwall 2016-12-01 18:49:38 +01:00
parent 4a9c753710
commit 84ade6fdc3
69 changed files with 1778 additions and 339 deletions

View file

@ -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()
}