* 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:
parent
777173f988
commit
65ccada280
41 changed files with 2389 additions and 159 deletions
|
|
@ -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()
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue