Use short configurable timeout for expectNoMessage #27212 (#27213)

* Use short configurable timeout for expectNoMessage #27212

* consistent with ActorTestKit in Typed

* fix config
This commit is contained in:
Patrik Nordwall 2019-06-26 15:35:20 +01:00 committed by Arnout Engelen
parent e567272766
commit 50e99e08d8
19 changed files with 78 additions and 67 deletions

View file

@ -429,7 +429,7 @@ There are a number of ways timeouts can be defined in Akka tests. The following
* `3.seconds` is third choice if not using testkit
* lower timeouts must come with a very good reason (e.g. Awaiting on a known to be "already completed" `Future`)
Special care should be given to `expectNoMsg` calls, which indeed will wait the entire timeout before continuing, therefore a shorter timeout should be used in those, for example `200` or `300.millis`.
Special care should be given to `expectNoMessage` calls, which indeed will wait the entire timeout before continuing, therefore a shorter timeout should be used in those, for example `200` or `300.millis`. Prefer the method without timeout parameter, which will use the configured `expect-no-message-default` timeout.
You can read up on `remaining` and friends in [TestKit.scala](https://github.com/akka/akka/blob/master/akka-testkit/src/main/scala/akka/testkit/TestKit.scala).

View file

@ -4,13 +4,10 @@
package akka.actor
import language.postfixOps
import org.scalatest.BeforeAndAfterEach
import akka.actor.Actor._
import akka.testkit._
import scala.concurrent.duration._
import java.util.concurrent.atomic._
import scala.concurrent.Await
import akka.pattern.ask
@ -68,7 +65,7 @@ class ActorLifeCycleSpec
expectMsg(("OK", id, 3))
restarter ! Kill
expectMsg(("postStop", id, 3))
expectNoMessage(1 seconds)
expectNoMessage()
system.stop(supervisor)
}
}
@ -100,7 +97,7 @@ class ActorLifeCycleSpec
expectMsg(("OK", id, 3))
restarter ! Kill
expectMsg(("postStop", id, 3))
expectNoMessage(1 seconds)
expectNoMessage()
system.stop(supervisor)
}
}
@ -117,7 +114,7 @@ class ActorLifeCycleSpec
expectMsg(("OK", id, 0))
system.stop(a)
expectMsg(("postStop", id, 0))
expectNoMessage(1 seconds)
expectNoMessage()
system.stop(supervisor)
}

View file

@ -4,8 +4,6 @@
package akka.actor
import language.postfixOps
import akka.testkit._
import scala.concurrent.duration._
import scala.concurrent.Await
@ -291,7 +289,7 @@ class ActorSelectionSpec extends AkkaSpec with DefaultTimeout {
case `c1` => lastSender
}
actors should ===(Set(c1, c2))
expectNoMessage(1 second)
expectNoMessage()
}
"drop messages which cannot be delivered" in {
@ -301,7 +299,7 @@ class ActorSelectionSpec extends AkkaSpec with DefaultTimeout {
case `c2` => lastSender
}
actors should ===(Seq(c21))
expectNoMessage(200.millis)
expectNoMessage()
}
"resolve one actor with explicit timeout" in {
@ -368,33 +366,33 @@ class ActorSelectionSpec extends AkkaSpec with DefaultTimeout {
system.actorSelection("/user/a/*").tell(Identify(1), probe.ref)
probe.receiveN(2).map { case ActorIdentity(1, r) => r }.toSet should ===(
Set[Option[ActorRef]](Some(b1), Some(b2)))
probe.expectNoMessage(200.millis)
probe.expectNoMessage()
system.actorSelection("/user/a/b1/*").tell(Identify(2), probe.ref)
probe.expectMsg(ActorIdentity(2, None))
system.actorSelection("/user/a/*/c").tell(Identify(3), probe.ref)
probe.expectMsg(ActorIdentity(3, Some(c)))
probe.expectNoMessage(200.millis)
probe.expectNoMessage()
system.actorSelection("/user/a/b2/*/d").tell(Identify(4), probe.ref)
probe.expectMsg(ActorIdentity(4, Some(d)))
probe.expectNoMessage(200.millis)
probe.expectNoMessage()
system.actorSelection("/user/a/*/*/d").tell(Identify(5), probe.ref)
probe.expectMsg(ActorIdentity(5, Some(d)))
probe.expectNoMessage(200.millis)
probe.expectNoMessage()
system.actorSelection("/user/a/*/c/*").tell(Identify(6), probe.ref)
probe.expectMsg(ActorIdentity(6, Some(d)))
probe.expectNoMessage(200.millis)
probe.expectNoMessage()
system.actorSelection("/user/a/b2/*/d/e").tell(Identify(7), probe.ref)
probe.expectMsg(ActorIdentity(7, None))
probe.expectNoMessage(200.millis)
probe.expectNoMessage()
system.actorSelection("/user/a/*/c/d/e").tell(Identify(8), probe.ref)
probe.expectNoMessage(500.millis)
probe.expectNoMessage()
}
"forward to selection" in {

View file

@ -79,7 +79,7 @@ class ActorSystemDispatchersSpec extends AkkaSpec(ConfigFactory.parseString("""
ref.tell("ping", probe.ref)
ecProbe.expectNoMessage(200.millis)
ecProbe.expectNoMessage()
probe.expectMsg(1.second, "ping")
} finally {
shutdown(system2)

View file

@ -249,7 +249,7 @@ class CoordinatedShutdownSpec
intercept[TimeoutException] {
Await.result(result, remainingOrDefault)
}
expectNoMessage(200.millis) // C not run
expectNoMessage() // C not run
}
"skip tasks in disabled phase" in {

View file

@ -42,14 +42,14 @@ class DeadLetterSupressionSpec extends AkkaSpec with ImplicitSender {
deadActor ! NormalMsg
deadListener.expectMsg(DeadLetter(NormalMsg, testActor, deadActor))
deadListener.expectNoMessage(200.millis)
deadListener.expectNoMessage()
suppressedListener.expectMsg(SuppressedDeadLetter(SuppressedMsg, testActor, system.deadLetters))
suppressedListener.expectNoMessage(200.millis)
suppressedListener.expectNoMessage()
allListener.expectMsg(SuppressedDeadLetter(SuppressedMsg, testActor, system.deadLetters))
allListener.expectMsg(DeadLetter(NormalMsg, testActor, deadActor))
allListener.expectNoMessage(200.millis)
allListener.expectNoMessage()
}
s"must suppress message from default dead-letters logging (sent to dead: ${Logging.simpleName(system.deadLetters)})" in {

View file

@ -6,7 +6,6 @@ package akka.actor
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import scala.concurrent.duration._
import akka.testkit.EventFilter
import akka.actor.dungeon.SerializationCheckFailedException
@ -100,7 +99,7 @@ class FunctionRefSpec extends AkkaSpec with ImplicitSender {
// needs to be something that fails when the deserialized form is not a FunctionRef
// this relies upon serialize-messages during tests
testActor ! DropForwarder(ref)
expectNoMessage(1.second)
expectNoMessage()
}
}
}

View file

@ -96,7 +96,7 @@ abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty(
bus.subscribe(subscriber, classifier)
bus.publish(event)
expectMsg(event)
expectNoMessage(1 second)
expectNoMessage()
bus.unsubscribe(subscriber, classifier)
}
@ -108,7 +108,7 @@ abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty(
expectMsg(event)
expectMsg(event)
expectMsg(event)
expectNoMessage(1 second)
expectNoMessage()
bus.unsubscribe(subscriber, classifier)
}
@ -136,14 +136,14 @@ abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty(
expectMsg(event)
bus.unsubscribe(subscriber, classifier)
bus.unsubscribe(otherSubscriber, otherClassifier)
expectNoMessage(1 second)
expectNoMessage()
}
"not publish the given event to a former subscriber" in {
bus.subscribe(subscriber, classifier)
bus.unsubscribe(subscriber, classifier)
bus.publish(event)
expectNoMessage(1 second)
expectNoMessage()
}
"cleanup subscriber" in {
@ -207,7 +207,7 @@ class ActorEventBusSpec(conf: Config) extends EventBusSpec("ActorEventBus", conf
expectUnsubscribedByUnsubscriber(p, subs)
bus.publish(m(2))
expectNoMessage(1 second)
expectNoMessage()
disposeSubscriber(system, subs)
disposeSubscriber(system, a1)
@ -256,7 +256,7 @@ class ActorEventBusSpec(conf: Config) extends EventBusSpec("ActorEventBus", conf
bus.unsubscribe(subs, a1)
bus.publish(m1(2))
expectNoMessage(1 second)
expectNoMessage()
bus.publish(m2(2))
expectMsg(m2(2))
@ -264,7 +264,7 @@ class ActorEventBusSpec(conf: Config) extends EventBusSpec("ActorEventBus", conf
expectUnregisterFromUnsubscriber(p, subs)
bus.publish(m1(3))
bus.publish(m2(3))
expectNoMessage(1 second)
expectNoMessage()
disposeSubscriber(system, subs)
disposeSubscriber(system, a1)

View file

@ -17,8 +17,6 @@ import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }
import akka.actor.typed.scaladsl.adapter._
import akka.{ actor => u }
import scala.concurrent.duration._
object ProbedBehavior {
def behavior(probe: u.ActorRef): Behavior[String] = {
Behaviors
@ -50,10 +48,11 @@ object UntypedSupervisingTypedSpec {
}
}
class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
class UntypedSupervisingTypedSpec
extends AkkaSpec("akka.actor.testkit.typed.expect-no-message-default = 50 ms")
with ImplicitSender {
implicit val typedActorSystem: ActorSystem[Nothing] = system.toTyped
val smallDuration = 50.millis
"An untyped actor system that spawns typed actors" should {
"default to stop for supervision" in {
@ -62,7 +61,7 @@ class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
watch(underTest.toUntyped)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage(smallDuration)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
}
@ -72,7 +71,7 @@ class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
watch(underTest.toUntyped)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage(smallDuration)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
}
@ -83,8 +82,8 @@ class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
watch(underTest.toUntyped)
underTest ! "throw"
probe.expectMsg(PreRestart)
probe.expectNoMessage(smallDuration)
expectNoMessage(smallDuration)
probe.expectNoMessage()
expectNoMessage()
}
"default to stop supervision (from context)" in {
@ -95,7 +94,7 @@ class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
watch(underTest.toUntyped)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage(smallDuration)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
}
@ -108,8 +107,8 @@ class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
watch(underTest.toUntyped)
underTest ! "throw"
probe.expectMsg(PreRestart)
probe.expectNoMessage(smallDuration)
expectNoMessage(smallDuration)
probe.expectNoMessage()
expectNoMessage()
}
"default to stop supervision for spawn anonymous (from context)" in {
@ -120,7 +119,7 @@ class UntypedSupervisingTypedSpec extends AkkaSpec with ImplicitSender {
watch(underTest.toUntyped)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage(smallDuration)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
}

View file

@ -4,9 +4,7 @@
package akka.cluster
import language.postfixOps
import scala.collection.immutable.SortedSet
import scala.concurrent.duration._
import org.scalatest.BeforeAndAfterEach
import akka.actor.Address
import akka.actor.PoisonPill
@ -129,7 +127,7 @@ class ClusterDomainEventPublisherSpec
memberSubscriber.expectMsg(MemberExited(bExiting))
memberSubscriber.expectMsg(MemberUp(cUp))
memberSubscriber.expectMsg(LeaderChanged(Some(a51Up.address)))
memberSubscriber.expectNoMessage(500 millis)
memberSubscriber.expectNoMessage()
}
"publish leader changed when old leader leaves and is removed" in {
@ -141,7 +139,7 @@ class ClusterDomainEventPublisherSpec
publisher ! PublishChanges(state7)
memberSubscriber.expectMsg(MemberExited(aExiting))
memberSubscriber.expectMsg(LeaderChanged(Some(cUp.address)))
memberSubscriber.expectNoMessage(500 millis)
memberSubscriber.expectNoMessage()
// at the removed member a an empty gossip is the last thing
publisher ! PublishChanges(emptyMembershipState)
memberSubscriber.expectMsg(MemberRemoved(aRemoved, Exiting))
@ -158,7 +156,7 @@ class ClusterDomainEventPublisherSpec
memberSubscriber.expectMsg(LeaderChanged(Some(a51Up.address)))
publisher ! PublishChanges(state5)
memberSubscriber.expectNoMessage(500 millis)
memberSubscriber.expectNoMessage()
}
"publish role leader changed" in {
@ -178,7 +176,7 @@ class ClusterDomainEventPublisherSpec
publisher ! Subscribe(subscriber.ref, InitialStateAsSnapshot, Set(classOf[ClusterDomainEvent]))
subscriber.expectMsgType[CurrentClusterState]
// but only to the new subscriber
memberSubscriber.expectNoMessage(500 millis)
memberSubscriber.expectNoMessage()
}
"send events corresponding to current state when subscribe" in {
@ -187,7 +185,7 @@ class ClusterDomainEventPublisherSpec
publisher ! Subscribe(subscriber.ref, InitialStateAsEvents, Set(classOf[MemberEvent], classOf[ReachabilityEvent]))
subscriber.receiveN(4).toSet should be(Set(MemberUp(aUp), MemberUp(cUp), MemberUp(dUp), MemberExited(bExiting)))
subscriber.expectMsg(UnreachableMember(dUp))
subscriber.expectNoMessage(500 millis)
subscriber.expectNoMessage()
}
"send datacenter reachability events" in {
@ -195,10 +193,10 @@ class ClusterDomainEventPublisherSpec
publisher ! PublishChanges(state9)
publisher ! Subscribe(subscriber.ref, InitialStateAsEvents, Set(classOf[DataCenterReachabilityEvent]))
subscriber.expectMsg(UnreachableDataCenter(OtherDataCenter))
subscriber.expectNoMessage(500 millis)
subscriber.expectNoMessage()
publisher ! PublishChanges(state10)
subscriber.expectMsg(ReachableDataCenter(OtherDataCenter))
subscriber.expectNoMessage(500 millis)
subscriber.expectNoMessage()
}
"support unsubscribe" in {
@ -207,7 +205,7 @@ class ClusterDomainEventPublisherSpec
subscriber.expectMsgType[CurrentClusterState]
publisher ! Unsubscribe(subscriber.ref, Some(classOf[MemberEvent]))
publisher ! PublishChanges(state3)
subscriber.expectNoMessage(500 millis)
subscriber.expectNoMessage()
// but memberSubscriber is still subscriber
memberSubscriber.expectMsg(MemberExited(bExiting))
memberSubscriber.expectMsg(MemberUp(cUp))
@ -219,10 +217,10 @@ class ClusterDomainEventPublisherSpec
subscriber.expectMsgType[CurrentClusterState]
publisher ! PublishChanges(state2)
subscriber.expectMsgType[SeenChanged]
subscriber.expectNoMessage(500 millis)
subscriber.expectNoMessage()
publisher ! PublishChanges(state3)
subscriber.expectMsgType[SeenChanged]
subscriber.expectNoMessage(500 millis)
subscriber.expectNoMessage()
}
"publish ClusterShuttingDown and Removed when stopped" in {

View file

@ -331,7 +331,7 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
enterBarrier("replicated-2")
// no OversizedPayloadException logging
errorLogProbe.expectNoMessage(100.millis)
errorLogProbe.expectNoMessage()
enterBarrierAfterTestStep()
}

View file

@ -219,7 +219,7 @@ When used without Cluster
akka.remote.warn-unsafe-watch-without-cluster = off
```
## Schedule periodically with fixed-delay vs. fixed-rate
### Schedule periodically with fixed-delay vs. fixed-rate
The `Scheduler.schedule` method has been deprecated in favor of selecting `scheduleWithFixedDelay` or
`scheduleAtFixedRate`.
@ -325,6 +325,11 @@ Configuration property:
akka.cluster.monitored-by-nr-of-members = 9
```
### TestKit
`expectNoMessage()` without timeout parameter is now using a new configuration property
`akka.test.expect-no-message-default` (short timeout) instead of `remainingOrDefault` (long timeout).
## Source incompatibilities
### StreamRefs

View file

@ -39,7 +39,7 @@ class DeviceGroupSpec extends AkkaSpec {
val groupActor = system.actorOf(DeviceGroup.props("group"))
groupActor.tell(DeviceManager.RequestTrackDevice("wrongGroup", "device1"), probe.ref)
probe.expectNoMessage(500.milliseconds)
probe.expectNoMessage()
}
//#device-group-test-registration

View file

@ -27,10 +27,10 @@ class DeviceSpec extends AkkaSpec {
val deviceActor = system.actorOf(Device.props("group", "device"))
deviceActor.tell(DeviceManager.RequestTrackDevice("wrongGroup", "device"), probe.ref)
probe.expectNoMessage(500.milliseconds)
probe.expectNoMessage()
deviceActor.tell(DeviceManager.RequestTrackDevice("group", "Wrongdevice"), probe.ref)
probe.expectNoMessage(500.milliseconds)
probe.expectNoMessage()
}
//#device-registration-tests

View file

@ -152,6 +152,7 @@ object TestPublisher {
/**
* Expect no messages.
* Waits for the default period configured as `akka.actor.testkit.expect-no-message-default`.
*/
def expectNoMessage(): Self = executeAfterSubscription {
probe.expectNoMessage()

View file

@ -19,6 +19,10 @@ akka {
# by default, will be dilated by the timefactor.
single-expect-default = 3s
# duration to wait in expectNoMessage by default,
# will be dilated by the timefactor.
expect-no-message-default = 100ms
# The timeout that is added as an implicit by DefaultTimeout trait
default-timeout = 5s

View file

@ -666,10 +666,12 @@ trait TestKitBase {
}
/**
* Same as `expectNoMsg(remainingOrDefault)`, but correctly treating the timeFactor.
* Assert that no message is received. Waits for the default period configured as
* `akka.test.expect-no-message-default`.
* That timeout is scaled using the configuration entry "akka.test.timefactor".
*/
@deprecated(message = "Use expectNoMessage instead", since = "2.5.5")
def expectNoMsg(): Unit = { expectNoMsg_internal(remainingOrDefault) }
def expectNoMsg(): Unit = expectNoMessage()
/**
* Assert that no message is received for the specified time.
@ -689,9 +691,12 @@ trait TestKitBase {
}
/**
* Same as `expectNoMessage(remainingOrDefault)`, but correctly treating the timeFactor.
* Assert that no message is received. Waits for the default period configured as
* `akka.test.expect-no-message-default`.
* That timeout is scaled using the configuration entry "akka.test.timefactor".
*/
def expectNoMessage(): Unit = { expectNoMsg_internal(remainingOrDefault) }
def expectNoMessage(): Unit =
expectNoMsg_internal(testKitSettings.ExpectNoMessageDefaultTimeout.dilated)
private def expectNoMsg_internal(max: FiniteDuration): Unit = {
val finish = System.nanoTime() + max.toNanos

View file

@ -22,6 +22,7 @@ class TestKitSettings(val config: Config) extends Extension {
.getDouble("akka.test.timefactor")
.requiring(tf => !tf.isInfinite && tf > 0, "akka.test.timefactor must be positive finite double")
val SingleExpectDefaultTimeout: FiniteDuration = config.getMillisDuration("akka.test.single-expect-default")
val ExpectNoMessageDefaultTimeout: FiniteDuration = config.getMillisDuration("akka.test.expect-no-message-default")
val TestEventFilterLeeway: FiniteDuration = config.getMillisDuration("akka.test.filter-leeway")
val DefaultTimeout: Timeout = Timeout(config.getMillisDuration("akka.test.default-timeout"))
}

View file

@ -632,13 +632,17 @@ class TestKit(system: ActorSystem) {
tp.expectMsgAnyClassOf(max.asScala, objs: _*).asInstanceOf[T]
/**
* Same as `expectNoMsg(remainingOrDefault)`, but correctly treating the timeFactor.
* Assert that no message is received. Waits for the default period configured as
* `akka.actor.testkit.expect-no-message-default`.
* That timeout is scaled using the configuration entry "akka.actor.testkit.typed.timefactor".
*/
@deprecated(message = "Use expectNoMessage instead", since = "2.5.10")
def expectNoMsg(): Unit = tp.expectNoMessage()
/**
* Same as `expectNoMessage(remainingOrDefault)`, but correctly treating the timeFactor.
* Assert that no message is received. Waits for the default period configured as
* `akka.actor.testkit.expect-no-message-default`.
* That timeout is scaled using the configuration entry "akka.actor.testkit.typed.timefactor".
*/
def expectNoMessage(): Unit = tp.expectNoMessage()