2019-03-28 13:24:46 +01:00
|
|
|
/*
|
2020-01-02 07:24:59 -05:00
|
|
|
* Copyright (C) 2019-2020 Lightbend Inc. <https://www.lightbend.com>
|
2019-03-28 13:24:46 +01:00
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
package akka.cluster.singleton
|
|
|
|
|
|
|
|
|
|
import java.util.concurrent.atomic.AtomicInteger
|
|
|
|
|
|
|
|
|
|
import scala.concurrent.Promise
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import scala.util.Success
|
|
|
|
|
|
2020-04-27 20:32:18 +08:00
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
|
Disable Java serialization by default, #22333 (#27285)
* akka.actor.allow-java-serialization = off
* Moved primitive (Long, Int, String, ByteString) serializers
from akka-remote to akka-actor since they had no dependency
and are useful also in local systems, e.g. persistence.
* e.g. needed for persistence-tck
* less allow-java-serialization=on in tests
* CborSerializable in Jackson/test module for ease of use
* JavaSerializable for Java serialization in tests, already in akka-testkit,
but misconfigured
* Made tests pass
* allow-java-serialization=on in akka-persistence
* allow-java-serialization=on in classic remoting tests
* JavaSerializable and CborSerializable in other remoting tests
* Added serialization for
* Boolean
* java.util.concurrent.TimeoutException, AskTimeoutException
* support for testing serialization with the inmem journal
* utility to verifySerialization, in SerializationTestKit
* remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static
* Effect() is factory in EventSourcedBehavior class
* test the account examples
* SharedLeveldbJournal.configToEnableJavaSerializationForTest
* support for exceptions from remote deployed child actors
* fallback to akka.remote.serialization.ThrowableNotSerializableException
if exception is not serializable when wrapped in system messages from
remote deployed child actors and Status.Failure messages
* it's implemented in `WrappedPayloadSupport.payloadBuilder`
* update reference documentation
* serialize-messages=off in most places, separate ticket for
improving or removing that feature
* migration guide, including description of rolling update
* fix 2.13 compiler error
* minor review feedback
2019-07-11 14:04:24 +02:00
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.ActorLogging
|
|
|
|
|
import akka.actor.ActorRef
|
|
|
|
|
import akka.actor.ExtendedActorSystem
|
|
|
|
|
import akka.actor.PoisonPill
|
|
|
|
|
import akka.actor.Props
|
|
|
|
|
import akka.cluster.Cluster
|
|
|
|
|
import akka.cluster.MemberStatus
|
2020-05-25 12:21:13 +02:00
|
|
|
import akka.coordination.lease.TestLease
|
|
|
|
|
import akka.coordination.lease.TestLeaseExt
|
Disable Java serialization by default, #22333 (#27285)
* akka.actor.allow-java-serialization = off
* Moved primitive (Long, Int, String, ByteString) serializers
from akka-remote to akka-actor since they had no dependency
and are useful also in local systems, e.g. persistence.
* e.g. needed for persistence-tck
* less allow-java-serialization=on in tests
* CborSerializable in Jackson/test module for ease of use
* JavaSerializable for Java serialization in tests, already in akka-testkit,
but misconfigured
* Made tests pass
* allow-java-serialization=on in akka-persistence
* allow-java-serialization=on in classic remoting tests
* JavaSerializable and CborSerializable in other remoting tests
* Added serialization for
* Boolean
* java.util.concurrent.TimeoutException, AskTimeoutException
* support for testing serialization with the inmem journal
* utility to verifySerialization, in SerializationTestKit
* remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static
* Effect() is factory in EventSourcedBehavior class
* test the account examples
* SharedLeveldbJournal.configToEnableJavaSerializationForTest
* support for exceptions from remote deployed child actors
* fallback to akka.remote.serialization.ThrowableNotSerializableException
if exception is not serializable when wrapped in system messages from
remote deployed child actors and Status.Failure messages
* it's implemented in `WrappedPayloadSupport.payloadBuilder`
* update reference documentation
* serialize-messages=off in most places, separate ticket for
improving or removing that feature
* migration guide, including description of rolling update
* fix 2.13 compiler error
* minor review feedback
2019-07-11 14:04:24 +02:00
|
|
|
import akka.testkit.AkkaSpec
|
|
|
|
|
import akka.testkit.TestException
|
|
|
|
|
import akka.testkit.TestProbe
|
|
|
|
|
|
2019-03-28 13:24:46 +01:00
|
|
|
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 = {
|
2019-04-15 17:40:26 +02:00
|
|
|
case msg =>
|
2019-03-28 13:24:46 +01:00
|
|
|
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)) {
|
2020-05-25 12:21:13 +02:00
|
|
|
import TestLease.{ AcquireReq, ReleaseReq }
|
2019-03-28 13:24:46 +01:00
|
|
|
|
|
|
|
|
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)
|
2020-04-27 17:31:16 +07:00
|
|
|
val nextResponse = Promise[Boolean]()
|
2019-03-28 13:24:46 +01:00
|
|
|
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)
|
Disable Java serialization by default, #22333 (#27285)
* akka.actor.allow-java-serialization = off
* Moved primitive (Long, Int, String, ByteString) serializers
from akka-remote to akka-actor since they had no dependency
and are useful also in local systems, e.g. persistence.
* e.g. needed for persistence-tck
* less allow-java-serialization=on in tests
* CborSerializable in Jackson/test module for ease of use
* JavaSerializable for Java serialization in tests, already in akka-testkit,
but misconfigured
* Made tests pass
* allow-java-serialization=on in akka-persistence
* allow-java-serialization=on in classic remoting tests
* JavaSerializable and CborSerializable in other remoting tests
* Added serialization for
* Boolean
* java.util.concurrent.TimeoutException, AskTimeoutException
* support for testing serialization with the inmem journal
* utility to verifySerialization, in SerializationTestKit
* remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static
* Effect() is factory in EventSourcedBehavior class
* test the account examples
* SharedLeveldbJournal.configToEnableJavaSerializationForTest
* support for exceptions from remote deployed child actors
* fallback to akka.remote.serialization.ThrowableNotSerializableException
if exception is not serializable when wrapped in system messages from
remote deployed child actors and Status.Failure messages
* it's implemented in `WrappedPayloadSupport.payloadBuilder`
* update reference documentation
* serialize-messages=off in most places, separate ticket for
improving or removing that feature
* migration guide, including description of rolling update
* fix 2.13 compiler error
* minor review feedback
2019-07-11 14:04:24 +02:00
|
|
|
testLease.initialPromise.failure(TestException("no lease for you"))
|
2019-03-28 13:24:46 +01:00
|
|
|
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)
|
2020-04-27 17:31:16 +07:00
|
|
|
val nextResponse = Promise[Boolean]()
|
2019-03-28 13:24:46 +01:00
|
|
|
testLease.setNextAcquireResult(nextResponse.future)
|
Disable Java serialization by default, #22333 (#27285)
* akka.actor.allow-java-serialization = off
* Moved primitive (Long, Int, String, ByteString) serializers
from akka-remote to akka-actor since they had no dependency
and are useful also in local systems, e.g. persistence.
* e.g. needed for persistence-tck
* less allow-java-serialization=on in tests
* CborSerializable in Jackson/test module for ease of use
* JavaSerializable for Java serialization in tests, already in akka-testkit,
but misconfigured
* Made tests pass
* allow-java-serialization=on in akka-persistence
* allow-java-serialization=on in classic remoting tests
* JavaSerializable and CborSerializable in other remoting tests
* Added serialization for
* Boolean
* java.util.concurrent.TimeoutException, AskTimeoutException
* support for testing serialization with the inmem journal
* utility to verifySerialization, in SerializationTestKit
* remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static
* Effect() is factory in EventSourcedBehavior class
* test the account examples
* SharedLeveldbJournal.configToEnableJavaSerializationForTest
* support for exceptions from remote deployed child actors
* fallback to akka.remote.serialization.ThrowableNotSerializableException
if exception is not serializable when wrapped in system messages from
remote deployed child actors and Status.Failure messages
* it's implemented in `WrappedPayloadSupport.payloadBuilder`
* update reference documentation
* serialize-messages=off in most places, separate ticket for
improving or removing that feature
* migration guide, including description of rolling update
* fix 2.13 compiler error
* minor review feedback
2019-07-11 14:04:24 +02:00
|
|
|
testLease.initialPromise.failure(TestException("no lease for you"))
|
2019-03-28 13:24:46 +01:00
|
|
|
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))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|