pekko/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonLeaseSpec.scala
Christopher Batey 65ccada280 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
2019-03-28 13:31:56 +01:00

195 lines
7.1 KiB
Scala

/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.cluster.singleton
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.{ Actor, ActorLogging, ActorRef, ExtendedActorSystem, PoisonPill, Props }
import akka.cluster.TestLease.{ AcquireReq, ReleaseReq }
import akka.cluster.{ Cluster, MemberStatus, TestLease, TestLeaseExt }
import akka.testkit.{ AkkaSpec, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.Success
class ImportantSingleton(lifeCycleProbe: ActorRef) extends Actor with ActorLogging {
override def preStart(): Unit = {
log.info("Important Singleton Starting")
lifeCycleProbe ! "preStart"
}
override def postStop(): Unit = {
log.info("Important Singleton Stopping")
lifeCycleProbe ! "postStop"
}
override def receive: Receive = {
case msg
sender() ! msg
}
}
class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = cluster
akka.cluster.singleton {
use-lease = "test-lease"
lease-retry-interval = 2000ms
}
""").withFallback(TestLease.config)) {
val cluster = Cluster(system)
val testLeaseExt = TestLeaseExt(system)
override protected def atStartup(): Unit = {
cluster.join(cluster.selfAddress)
awaitAssert {
cluster.selfMember.status shouldEqual MemberStatus.Up
}
}
def extSystem: ExtendedActorSystem = system.asInstanceOf[ExtendedActorSystem]
val counter = new AtomicInteger()
def nextName() = s"important-${counter.getAndIncrement()}"
val shortDuration = 50.millis
val leaseOwner = cluster.selfMember.address.hostPort
def nextSettings() = ClusterSingletonManagerSettings(system).withSingletonName(nextName())
def leaseNameFor(settings: ClusterSingletonManagerSettings): String =
s"ClusterSingletonLeaseSpec-singleton-akka://ClusterSingletonLeaseSpec/user/${settings.singletonName}"
"A singleton with lease" should {
"not start until lease is available" in {
val probe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(probe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
probe.expectNoMessage(shortDuration)
testLease.initialPromise.complete(Success(true))
probe.expectMsg("preStart")
}
"do not start if lease acquire returns false" in {
val probe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(probe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
probe.expectNoMessage(shortDuration)
testLease.initialPromise.complete(Success(false))
probe.expectNoMessage(shortDuration)
}
"retry trying to get lease if acquire returns false" in {
val singletonProbe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(singletonProbe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
val nextResponse = Promise[Boolean]
testLease.setNextAcquireResult(nextResponse.future)
testLease.initialPromise.complete(Success(false))
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
nextResponse.complete(Success(true))
singletonProbe.expectMsg("preStart")
}
"do not start if lease acquire fails" in {
val probe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(probe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
probe.expectNoMessage(shortDuration)
testLease.initialPromise.failure(new RuntimeException("no lease for you"))
probe.expectNoMessage(shortDuration)
}
"retry trying to get lease if acquire returns fails" in {
val singletonProbe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(singletonProbe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
val nextResponse = Promise[Boolean]
testLease.setNextAcquireResult(nextResponse.future)
testLease.initialPromise.failure(new RuntimeException("no lease for you"))
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
nextResponse.complete(Success(true))
singletonProbe.expectMsg("preStart")
}
"stop singleton if the lease fails periodic check" in {
val lifecycleProbe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(lifecycleProbe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
}
testLease.probe.expectMsg(AcquireReq(leaseOwner))
testLease.initialPromise.complete(Success(true))
lifecycleProbe.expectMsg("preStart")
val callback = testLease.getCurrentCallback()
callback(None)
lifecycleProbe.expectMsg("postStop")
testLease.probe.expectMsg(ReleaseReq(leaseOwner))
// should try and reacquire lease
testLease.probe.expectMsg(AcquireReq(leaseOwner))
lifecycleProbe.expectMsg("preStart")
}
"release lease when leaving oldest" in {
val singletonProbe = TestProbe()
val settings = nextSettings()
system.actorOf(
ClusterSingletonManager.props(Props(new ImportantSingleton(singletonProbe.ref)), PoisonPill, settings),
settings.singletonName)
val testLease = awaitAssert {
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
singletonProbe.expectNoMessage(shortDuration)
testLease.probe.expectMsg(AcquireReq(leaseOwner))
testLease.initialPromise.complete(Success(true))
singletonProbe.expectMsg("preStart")
cluster.leave(cluster.selfAddress)
testLease.probe.expectMsg(ReleaseReq(leaseOwner))
}
}
}