Lease API + use in cluster singleton and sharding, #26480 (#26629)

* lease api
* Cluster singleton manager with lease
* Refactor OldestData to use option for actor reference
* Sharding with lease
* Docs for singleton and sharding lease + config for sharding lease
* Have ddata shard wait until lease is acquired before getting state
This commit is contained in:
Christopher Batey 2019-03-28 13:24:46 +01:00 committed by Patrik Nordwall
parent 777173f988
commit 65ccada280
41 changed files with 2389 additions and 159 deletions

View file

@ -5,10 +5,10 @@
package akka.cluster.singleton
import com.typesafe.config.Config
import scala.concurrent.duration._
import scala.collection.immutable
import scala.concurrent.Future
import akka.actor.Actor
import akka.actor.Deploy
import akka.actor.ActorSystem
@ -19,22 +19,22 @@ import akka.actor.DeadLetterSuppression
import akka.actor.FSM
import akka.actor.Props
import akka.actor.Terminated
import akka.cluster.Cluster
import akka.cluster._
import akka.cluster.ClusterEvent._
import akka.cluster.Member
import akka.cluster.MemberStatus
import akka.AkkaException
import akka.actor.NoSerializationVerificationNeeded
import akka.cluster.UniqueAddress
import akka.cluster.ClusterEvent
import scala.concurrent.Promise
import akka.pattern.pipe
import akka.util.JavaDurationConverters._
import scala.concurrent.Promise
import akka.Done
import akka.actor.CoordinatedShutdown
import akka.annotation.DoNotInherit
import akka.annotation.{ ApiMayChange, DoNotInherit }
import akka.pattern.ask
import akka.util.Timeout
import akka.cluster.ClusterSettings
import akka.coordination.lease.scaladsl.{ Lease, LeaseProvider }
import scala.util.control.NonFatal
object ClusterSingletonManagerSettings {
@ -52,12 +52,19 @@ object ClusterSingletonManagerSettings {
* Create settings from a configuration with the same layout as
* the default configuration `akka.cluster.singleton`.
*/
def apply(config: Config): ClusterSingletonManagerSettings =
def apply(config: Config): ClusterSingletonManagerSettings = {
val lease = config.getString("use-lease") match {
case s if s.isEmpty None
case leaseConfigPath =>
Some(new ClusterLeaseSettings(leaseConfigPath, config.getDuration("lease-retry-interval").asScala))
}
new ClusterSingletonManagerSettings(
singletonName = config.getString("singleton-name"),
role = roleOption(config.getString("role")),
removalMargin = Duration.Zero, // defaults to ClusterSettins.DownRemovalMargin
handOverRetryInterval = config.getDuration("hand-over-retry-interval", MILLISECONDS).millis)
removalMargin = Duration.Zero, // defaults to ClusterSettings.DownRemovalMargin
handOverRetryInterval = config.getDuration("hand-over-retry-interval", MILLISECONDS).millis,
lease)
}
/**
* Java API: Create settings from the default configuration
@ -98,14 +105,25 @@ object ClusterSingletonManagerSettings {
* retried with this interval until the previous oldest confirms that the hand
* over has started or the previous oldest member is removed from the cluster
* (+ `removalMargin`).
*
* @param leaseSettings LeaseSettings for acquiring before creating the singleton actor
*/
final class ClusterSingletonManagerSettings(
val singletonName: String,
val role: Option[String],
val removalMargin: FiniteDuration,
val handOverRetryInterval: FiniteDuration)
val handOverRetryInterval: FiniteDuration,
val leaseSettings: Option[ClusterLeaseSettings])
extends NoSerializationVerificationNeeded {
// bin compat for akka 2.5.21
def this(
singletonName: String,
role: Option[String],
removalMargin: FiniteDuration,
handOverRetryInterval: FiniteDuration) =
this(singletonName, role, removalMargin, handOverRetryInterval, None)
def withSingletonName(name: String): ClusterSingletonManagerSettings = copy(singletonName = name)
def withRole(role: String): ClusterSingletonManagerSettings =
@ -119,12 +137,16 @@ final class ClusterSingletonManagerSettings(
def withHandOverRetryInterval(retryInterval: FiniteDuration): ClusterSingletonManagerSettings =
copy(handOverRetryInterval = retryInterval)
def withLeaseSettings(leaseSettings: ClusterLeaseSettings): ClusterSingletonManagerSettings =
copy(leaseSettings = Some(leaseSettings))
private def copy(
singletonName: String = singletonName,
role: Option[String] = role,
removalMargin: FiniteDuration = removalMargin,
handOverRetryInterval: FiniteDuration = handOverRetryInterval): ClusterSingletonManagerSettings =
new ClusterSingletonManagerSettings(singletonName, role, removalMargin, handOverRetryInterval)
handOverRetryInterval: FiniteDuration = handOverRetryInterval,
leaseSettings: Option[ClusterLeaseSettings] = leaseSettings): ClusterSingletonManagerSettings =
new ClusterSingletonManagerSettings(singletonName, role, removalMargin, handOverRetryInterval, leaseSettings)
}
/**
@ -189,10 +211,12 @@ object ClusterSingletonManager {
final case class HandOverRetry(count: Int)
final case class TakeOverRetry(count: Int)
final case object LeaseRetry
case object Cleanup
case object StartOldestChangedBuffer
case object Start extends State
case object AcquiringLease extends State
case object Oldest extends State
case object Younger extends State
case object BecomingOldest extends State
@ -205,21 +229,19 @@ object ClusterSingletonManager {
case object Uninitialized extends Data
final case class YoungerData(oldestOption: Option[UniqueAddress]) extends Data
final case class BecomingOldestData(previousOldestOption: Option[UniqueAddress]) extends Data
final case class OldestData(singleton: ActorRef, singletonTerminated: Boolean = false) extends Data
final case class WasOldestData(
singleton: ActorRef,
singletonTerminated: Boolean,
newOldestOption: Option[UniqueAddress])
extends Data
final case class OldestData(singleton: Option[ActorRef]) extends Data
final case class WasOldestData(singleton: Option[ActorRef], newOldestOption: Option[UniqueAddress]) extends Data
final case class HandingOverData(singleton: ActorRef, handOverTo: Option[ActorRef]) extends Data
final case class StoppingData(singleton: ActorRef) extends Data
case object EndData extends Data
final case class DelayedMemberRemoved(member: Member)
case object SelfExiting
case class AcquiringLeaseData(leaseRequestInProgress: Boolean, singleton: Option[ActorRef]) extends Data
val HandOverRetryTimer = "hand-over-retry"
val TakeOverRetryTimer = "take-over-retry"
val CleanupTimer = "cleanup"
val LeaseRetryTimer = "lease-retry"
object OldestChangedBuffer {
@ -236,8 +258,14 @@ object ClusterSingletonManager {
final case class OldestChanged(oldest: Option[UniqueAddress])
}
final case class AcquireLeaseResult(holdingLease: Boolean) extends DeadLetterSuppression
final case class ReleaseLeaseResult(released: Boolean) extends DeadLetterSuppression
final case class AcquireLeaseFailure(t: Throwable) extends DeadLetterSuppression
final case class ReleaseLeaseFailure(t: Throwable) extends DeadLetterSuppression
final case class LeaseLost(reason: Option[Throwable]) extends DeadLetterSuppression
/**
* Notifications of member events that track oldest member is tunneled
* Notifications of member events that track oldest member are tunneled
* via this actor (child of ClusterSingletonManager) to be able to deliver
* one change at a time. Avoiding simultaneous changes simplifies
* the process in ClusterSingletonManager. ClusterSingletonManager requests
@ -457,6 +485,17 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
role.forall(cluster.selfRoles.contains),
s"This cluster member [${cluster.selfAddress}] doesn't have the role [$role]")
private val singletonLeaseName = s"${context.system.name}-singleton-${self.path}"
val lease: Option[Lease] = settings.leaseSettings.map(
settings =>
LeaseProvider(context.system)
.getLease(singletonLeaseName, settings.leaseImplementation, cluster.selfAddress.hostPort))
val leaseRetryInterval: FiniteDuration = settings.leaseSettings match {
case Some(s) => s.leaseRetryInterval
case None => 5.seconds // won't be used
}
val removalMargin =
if (settings.removalMargin <= Duration.Zero) cluster.downingProvider.downRemovalMargin
else settings.removalMargin
@ -515,6 +554,9 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
def logInfo(template: String, arg1: Any, arg2: Any): Unit =
if (LogInfo) log.info(template, arg1, arg2)
def logInfo(template: String, arg1: Any, arg2: Any, arg3: Any): Unit =
if (LogInfo) log.info(template, arg1, arg2, arg3)
override def preStart(): Unit = {
super.preStart()
require(!cluster.isTerminated, "Cluster node must not be terminated")
@ -557,7 +599,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
oldestChangedReceived = true
if (oldestOption == selfUniqueAddressOption && safeToBeOldest)
// oldest immediately
gotoOldest()
tryGoToOldest()
else if (oldestOption == selfUniqueAddressOption)
goto(BecomingOldest).using(BecomingOldestData(None))
else
@ -570,8 +612,8 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
if (oldestOption == selfUniqueAddressOption) {
logInfo("Younger observed OldestChanged: [{} -> myself]", previousOldestOption.map(_.address))
previousOldestOption match {
case None => gotoOldest()
case Some(prev) if removed.contains(prev) => gotoOldest()
case None => tryGoToOldest()
case Some(prev) if removed.contains(prev) => tryGoToOldest()
case Some(prev) =>
peer(prev.address) ! HandOverToMe
goto(BecomingOldest).using(BecomingOldestData(previousOldestOption))
@ -620,7 +662,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Event(HandOverDone, BecomingOldestData(Some(previousOldest))) =>
if (sender().path.address == previousOldest.address)
gotoOldest()
tryGoToOldest()
else {
logInfo(
"Ignoring HandOverDone in BecomingOldest from [{}]. Expected previous oldest [{}]",
@ -645,7 +687,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
if m.uniqueAddress == previousOldest =>
logInfo("Previous oldest [{}] removed", previousOldest.address)
addRemoved(m.uniqueAddress)
gotoOldest()
tryGoToOldest()
case Event(TakeOverFromMe, BecomingOldestData(previousOldestOption)) =>
val senderAddress = sender().path.address
@ -682,7 +724,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
// can't send HandOverToMe, previousOldest unknown for new node (or restart)
// previous oldest might be down or removed, so no TakeOverFromMe message is received
logInfo("Timeout in BecomingOldest. Previous oldest unknown, removed and no TakeOver request.")
gotoOldest()
tryGoToOldest()
} else if (cluster.isTerminated)
stop()
else
@ -698,47 +740,109 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
self ! DelayedMemberRemoved(m)
}
def gotoOldest(): State = {
def tryAcquireLease() = {
import context.dispatcher
pipe(lease.get.acquire(reason => self ! LeaseLost(reason)).map[Any](AcquireLeaseResult).recover {
case NonFatal(t) => AcquireLeaseFailure(t)
}).to(self)
goto(AcquiringLease).using(AcquiringLeaseData(leaseRequestInProgress = true, None))
}
// Try and go to oldest, taking the lease if needed
def tryGoToOldest(): State = {
// check if lease
lease match {
case None =>
goToOldest()
case Some(_) =>
logInfo("Trying to acquire lease before starting singleton")
tryAcquireLease()
}
}
when(AcquiringLease) {
case Event(AcquireLeaseResult(result), _) =>
logInfo("Acquire lease result {}", result)
if (result) {
goToOldest()
} else {
setTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
stay.using(AcquiringLeaseData(leaseRequestInProgress = false, None))
}
case Event(Terminated(ref), AcquiringLeaseData(_, Some(singleton))) if ref == singleton =>
logInfo("Singleton actor terminated. Trying to acquire lease again before re-creating.")
// tryAcquireLease sets the state to None for singleton actor
tryAcquireLease()
case Event(AcquireLeaseFailure(t), _) =>
log.error(t, "failed to get lease (will be retried)")
setTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
stay.using(AcquiringLeaseData(leaseRequestInProgress = false, None))
case Event(LeaseRetry, _) =>
// If lease was lost (so previous state was oldest) then we don't try and get the lease
// until the old singleton instance has been terminated so we know there isn't an
// instance in this case
tryAcquireLease()
case Event(OldestChanged(oldestOption), AcquiringLeaseData(_, singleton)) =>
handleOldestChanged(singleton, oldestOption)
case Event(HandOverToMe, AcquiringLeaseData(_, singleton)) =>
gotoHandingOver(singleton, Some(sender()))
case Event(TakeOverFromMe, _) =>
// already oldest, so confirm and continue like that
sender() ! HandOverToMe
stay
case Event(SelfExiting, _) =>
selfMemberExited()
// complete memberExitingProgress when handOverDone
sender() ! Done // reply to ask
stay
case Event(MemberDowned(m), _) if m.uniqueAddress == cluster.selfUniqueAddress =>
logInfo("Self downed, stopping ClusterSingletonManager")
stop()
}
def goToOldest(): State = {
val singleton = context.watch(context.actorOf(singletonProps, singletonName))
logInfo("Singleton manager starting singleton actor [{}]", singleton.path)
goto(Oldest).using(OldestData(singleton))
goto(Oldest).using(OldestData(Some(singleton)))
}
def handleOldestChanged(singleton: Option[ActorRef], oldestOption: Option[UniqueAddress]) = {
oldestChangedReceived = true
logInfo("{} observed OldestChanged: [{} -> {}]", stateName, cluster.selfAddress, oldestOption.map(_.address))
oldestOption match {
case Some(a) if a == cluster.selfUniqueAddress =>
// already oldest
stay
case Some(a) if !selfExited && removed.contains(a) =>
// The member removal was not completed and the old removed node is considered
// oldest again. Safest is to terminate the singleton instance and goto Younger.
// This node will become oldest again when the other is removed again.
gotoHandingOver(singleton, None)
case Some(a) =>
// send TakeOver request in case the new oldest doesn't know previous oldest
peer(a.address) ! TakeOverFromMe
setTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval, repeat = false)
goto(WasOldest).using(WasOldestData(singleton, newOldestOption = Some(a)))
case None =>
// new oldest will initiate the hand-over
setTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval, repeat = false)
goto(WasOldest).using(WasOldestData(singleton, newOldestOption = None))
}
}
when(Oldest) {
case Event(OldestChanged(oldestOption), OldestData(singleton, singletonTerminated)) =>
oldestChangedReceived = true
logInfo("Oldest observed OldestChanged: [{} -> {}]", cluster.selfAddress, oldestOption.map(_.address))
oldestOption match {
case Some(a) if a == cluster.selfUniqueAddress =>
// already oldest
stay
case Some(a) if !selfExited && removed.contains(a) =>
// The member removal was not completed and the old removed node is considered
// oldest again. Safest is to terminate the singleton instance and goto Younger.
// This node will become oldest again when the other is removed again.
gotoHandingOver(singleton, singletonTerminated, None)
case Some(a) =>
// send TakeOver request in case the new oldest doesn't know previous oldest
peer(a.address) ! TakeOverFromMe
setTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval, repeat = false)
goto(WasOldest).using(WasOldestData(singleton, singletonTerminated, newOldestOption = Some(a)))
case None =>
// new oldest will initiate the hand-over
setTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval, repeat = false)
goto(WasOldest).using(WasOldestData(singleton, singletonTerminated, newOldestOption = None))
}
case Event(HandOverToMe, OldestData(singleton, singletonTerminated)) =>
gotoHandingOver(singleton, singletonTerminated, Some(sender()))
case Event(OldestChanged(oldestOption), OldestData(singleton)) =>
handleOldestChanged(singleton, oldestOption)
case Event(HandOverToMe, OldestData(singleton)) =>
gotoHandingOver(singleton, Some(sender()))
case Event(TakeOverFromMe, _) =>
// already oldest, so confirm and continue like that
sender() ! HandOverToMe
stay
case Event(Terminated(ref), d @ OldestData(singleton, _)) if ref == singleton =>
case Event(Terminated(ref), d @ OldestData(Some(singleton))) if ref == singleton =>
logInfo("Singleton actor [{}] was terminated", singleton.path)
stay.using(d.copy(singletonTerminated = true))
stay.using(d.copy(singleton = None))
case Event(SelfExiting, _) =>
selfMemberExited()
@ -746,22 +850,34 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
sender() ! Done // reply to ask
stay
case Event(MemberDowned(m), OldestData(singleton, singletonTerminated))
if m.uniqueAddress == cluster.selfUniqueAddress =>
if (singletonTerminated) {
logInfo("Self downed, stopping ClusterSingletonManager")
stop()
} else {
logInfo("Self downed, stopping")
gotoStopping(singleton)
case Event(MemberDowned(m), OldestData(singleton)) if m.uniqueAddress == cluster.selfUniqueAddress =>
singleton match {
case Some(s) =>
logInfo("Self downed, stopping")
gotoStopping(s)
case None =>
logInfo("Self downed, stopping ClusterSingletonManager")
stop()
}
case Event(LeaseLost(reason), OldestData(singleton)) =>
log.warning("Lease has been lost. Reason: {}. Terminating singleton and trying to re-acquire lease", reason)
singleton match {
case Some(s) =>
s ! terminationMessage
goto(AcquiringLease).using(AcquiringLeaseData(leaseRequestInProgress = false, singleton))
case None =>
tryAcquireLease()
}
}
when(WasOldest) {
case Event(TakeOverRetry(count), WasOldestData(singleton, singletonTerminated, newOldestOption)) =>
case Event(TakeOverRetry(count), WasOldestData(singleton, newOldestOption)) =>
if ((cluster.isTerminated || selfExited) && (newOldestOption.isEmpty || count > maxTakeOverRetries)) {
if (singletonTerminated) stop()
else gotoStopping(singleton)
singleton match {
case Some(s) => gotoStopping(s)
case None => stop()
}
} else if (count <= maxTakeOverRetries) {
if (maxTakeOverRetries - count <= 3)
logInfo("Retry [{}], sending TakeOverFromMe to [{}]", count, newOldestOption.map(_.address))
@ -773,21 +889,20 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
} else
throw new ClusterSingletonManagerIsStuck(s"Expected hand-over to [$newOldestOption] never occurred")
case Event(HandOverToMe, WasOldestData(singleton, singletonTerminated, _)) =>
gotoHandingOver(singleton, singletonTerminated, Some(sender()))
case Event(HandOverToMe, WasOldestData(singleton, _)) =>
gotoHandingOver(singleton, Some(sender()))
case Event(MemberRemoved(m, _), _) if m.uniqueAddress == cluster.selfUniqueAddress && !selfExited =>
logInfo("Self removed, stopping ClusterSingletonManager")
stop()
case Event(MemberRemoved(m, _), WasOldestData(singleton, singletonTerminated, Some(newOldest)))
case Event(MemberRemoved(m, _), WasOldestData(singleton, Some(newOldest)))
if !selfExited && m.uniqueAddress == newOldest =>
addRemoved(m.uniqueAddress)
gotoHandingOver(singleton, singletonTerminated, None)
gotoHandingOver(singleton, None)
case Event(Terminated(ref), d @ WasOldestData(singleton, _, _)) if ref == singleton =>
logInfo("Singleton actor [{}] was terminated", singleton.path)
stay.using(d.copy(singletonTerminated = true))
case Event(Terminated(ref), d @ WasOldestData(singleton, _)) if singleton.contains(ref) =>
logInfo("Singleton actor [{}] was terminated", ref.path)
stay.using(d.copy(singleton = None))
case Event(SelfExiting, _) =>
selfMemberExited()
@ -795,34 +910,34 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
sender() ! Done // reply to ask
stay
case Event(MemberDowned(m), OldestData(singleton, singletonTerminated))
if m.uniqueAddress == cluster.selfUniqueAddress =>
if (singletonTerminated) {
logInfo("Self downed, stopping ClusterSingletonManager")
stop()
} else {
logInfo("Self downed, stopping")
gotoStopping(singleton)
case Event(MemberDowned(m), WasOldestData(singleton, _)) if m.uniqueAddress == cluster.selfUniqueAddress =>
singleton match {
case None =>
logInfo("Self downed, stopping ClusterSingletonManager")
stop()
case Some(s) =>
logInfo("Self downed, stopping")
gotoStopping(s)
}
}
def gotoHandingOver(singleton: ActorRef, singletonTerminated: Boolean, handOverTo: Option[ActorRef]): State = {
if (singletonTerminated) {
handOverDone(handOverTo)
} else {
handOverTo.foreach { _ ! HandOverInProgress }
logInfo("Singleton manager stopping singleton actor [{}]", singleton.path)
singleton ! terminationMessage
goto(HandingOver).using(HandingOverData(singleton, handOverTo))
def gotoHandingOver(singleton: Option[ActorRef], handOverTo: Option[ActorRef]): State = {
singleton match {
case None =>
handOverDone(handOverTo)
case Some(s) =>
handOverTo.foreach { _ ! HandOverInProgress }
logInfo("Singleton manager stopping singleton actor [{}]", s.path)
s ! terminationMessage
goto(HandingOver).using(HandingOverData(s, handOverTo))
}
}
when(HandingOver) {
case (Event(Terminated(ref), HandingOverData(singleton, handOverTo))) if ref == singleton =>
case Event(Terminated(ref), HandingOverData(singleton, handOverTo)) if ref == singleton =>
handOverDone(handOverTo)
case Event(HandOverToMe, HandingOverData(singleton, handOverTo)) if handOverTo == Some(sender()) =>
case Event(HandOverToMe, HandingOverData(_, handOverTo)) if handOverTo.contains(sender()) =>
// retry
sender() ! HandOverInProgress
stay
@ -855,7 +970,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
}
when(Stopping) {
case (Event(Terminated(ref), StoppingData(singleton))) if ref == singleton =>
case Event(Terminated(ref), StoppingData(singleton)) if ref == singleton =>
logInfo("Singleton actor [{}] was terminated", singleton.path)
stop()
}
@ -901,6 +1016,20 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
if (m.uniqueAddress == cluster.selfUniqueAddress)
logInfo("Self downed, waiting for removal")
stay
case Event(ReleaseLeaseFailure(t), _) =>
log.error(
t,
"Failed to release lease. Singleton may not be able to run on another node until lease timeout occurs")
stay
case Event(ReleaseLeaseResult(released), _) =>
if (released) {
logInfo("Lease released")
} else {
// TODO we could retry
log.error(
"Failed to release lease. Singleton may not be able to run on another node until lease timeout occurs")
}
stay
}
onTransition {
@ -916,6 +1045,29 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case WasOldest -> _ => cancelTimer(TakeOverRetryTimer)
}
onTransition {
case (AcquiringLease, to) if to != Oldest =>
stateData match {
case AcquiringLeaseData(true, _) =>
logInfo("Releasing lease as leaving AcquiringLease going to [{}]", to)
import context.dispatcher
lease.foreach(l =>
pipe(l.release().map[Any](ReleaseLeaseResult).recover {
case t => ReleaseLeaseFailure(t)
}).to(self))
case _ =>
}
}
onTransition {
case Oldest -> _ =>
lease.foreach { l =>
logInfo("Releasing lease as leaving Oldest")
import context.dispatcher
pipe(l.release().map(ReleaseLeaseResult)).to(self)
}
}
onTransition {
case _ -> (Younger | Oldest) => getNextOldestChanged()
}