2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2019-01-02 18:55:26 +08:00
|
|
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
2011-04-27 15:00:41 +02:00
|
|
|
*/
|
|
|
|
|
|
2011-04-16 22:20:04 +02:00
|
|
|
package akka.testkit
|
|
|
|
|
|
|
|
|
|
import akka.actor._
|
2011-11-13 20:38:14 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicLong
|
2019-04-16 18:33:38 +02:00
|
|
|
|
2011-12-14 17:26:18 +01:00
|
|
|
import akka.dispatch._
|
2019-04-16 18:33:38 +02:00
|
|
|
|
2012-06-29 16:06:26 +02:00
|
|
|
import scala.concurrent.Await
|
2012-07-24 12:12:06 +02:00
|
|
|
import scala.reflect.ClassTag
|
2012-01-18 10:18:51 +01:00
|
|
|
import akka.pattern.ask
|
2019-04-16 18:33:38 +02:00
|
|
|
import com.github.ghik.silencer.silent
|
2011-04-27 15:00:41 +02:00
|
|
|
|
2011-04-16 22:20:04 +02:00
|
|
|
/**
|
|
|
|
|
* This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it
|
|
|
|
|
* overrides the dispatcher to CallingThreadDispatcher and sets the receiveTimeout to None. Otherwise,
|
|
|
|
|
* it acts just like a normal ActorRef. You may retrieve a reference to the underlying actor to test internal logic.
|
|
|
|
|
*
|
|
|
|
|
* @since 1.1
|
|
|
|
|
*/
|
2019-04-16 18:33:38 +02:00
|
|
|
@silent // 'early initializers' are deprecated on 2.13 and will be replaced with trait parameters on 2.14. https://github.com/akka/akka/issues/26753
|
2019-03-11 10:38:24 +01:00
|
|
|
class TestActorRef[T <: Actor](_system: ActorSystem, _props: Props, _supervisor: ActorRef, name: String) extends {
|
|
|
|
|
val props =
|
|
|
|
|
_props.withDispatcher(
|
|
|
|
|
if (_props.deploy.dispatcher == Deploy.NoDispatcherGiven) CallingThreadDispatcher.Id
|
|
|
|
|
else _props.dispatcher)
|
|
|
|
|
val dispatcher = _system.dispatchers.lookup(props.dispatcher)
|
|
|
|
|
private val disregard = _supervisor match {
|
|
|
|
|
case l: LocalActorRef => l.underlying.reserveChild(name)
|
|
|
|
|
case r: RepointableActorRef =>
|
|
|
|
|
r.underlying match {
|
|
|
|
|
case _: UnstartedCell =>
|
|
|
|
|
throw new IllegalStateException(
|
|
|
|
|
"cannot attach a TestActor to an unstarted top-level actor, ensure that it is started by sending a message and observing the reply")
|
|
|
|
|
case c: ActorCell => c.reserveChild(name)
|
|
|
|
|
case o =>
|
|
|
|
|
_system.log.error(
|
|
|
|
|
"trying to attach child {} to unknown type of supervisor cell {}, this is not going to end well",
|
|
|
|
|
name,
|
|
|
|
|
o.getClass)
|
2012-08-15 15:25:43 +02:00
|
|
|
}
|
2019-03-11 10:38:24 +01:00
|
|
|
case s =>
|
2019-03-13 10:56:20 +01:00
|
|
|
_system.log.error(
|
|
|
|
|
"trying to attach child {} to unknown type of supervisor {}, this is not going to end well",
|
|
|
|
|
name,
|
|
|
|
|
s.getClass)
|
2019-03-11 10:38:24 +01:00
|
|
|
}
|
2019-03-13 10:56:20 +01:00
|
|
|
} with LocalActorRef(
|
|
|
|
|
_system.asInstanceOf[ActorSystemImpl],
|
|
|
|
|
props,
|
|
|
|
|
dispatcher,
|
|
|
|
|
_system.mailboxes.getMailboxType(props, dispatcher.configurator.config),
|
|
|
|
|
_supervisor.asInstanceOf[InternalActorRef],
|
|
|
|
|
_supervisor.path / name) {
|
2011-12-10 16:56:53 +01:00
|
|
|
|
2012-10-15 13:33:39 +02:00
|
|
|
// we need to start ourselves since the creation of an actor has been split into initialization and starting
|
|
|
|
|
underlying.start()
|
|
|
|
|
|
2012-01-31 21:38:20 +01:00
|
|
|
import TestActorRef.InternalGetActor
|
2011-12-10 16:56:53 +01:00
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
protected override def newActorCell(
|
|
|
|
|
system: ActorSystemImpl,
|
|
|
|
|
ref: InternalActorRef,
|
|
|
|
|
props: Props,
|
|
|
|
|
dispatcher: MessageDispatcher,
|
|
|
|
|
supervisor: InternalActorRef): ActorCell =
|
2013-06-01 21:58:34 +02:00
|
|
|
new ActorCell(system, ref, props, dispatcher, supervisor) {
|
2018-07-25 20:38:27 +09:00
|
|
|
override def autoReceiveMessage(msg: Envelope): Unit = {
|
2011-12-10 16:56:53 +01:00
|
|
|
msg.message match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case InternalGetActor => sender() ! actor
|
|
|
|
|
case _ => super.autoReceiveMessage(msg)
|
2011-12-10 16:56:53 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2011-04-21 21:31:24 +02:00
|
|
|
/**
|
|
|
|
|
* Directly inject messages into actor receive behavior. Any exceptions
|
|
|
|
|
* thrown will be available to you, while still being able to use
|
2011-12-10 16:56:53 +01:00
|
|
|
* become/unbecome.
|
2011-04-21 21:31:24 +02:00
|
|
|
*/
|
2012-06-04 17:07:44 +02:00
|
|
|
def receive(o: Any): Unit = receive(o, underlying.system.deadLetters)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Directly inject messages into actor receive behavior. Any exceptions
|
|
|
|
|
* thrown will be available to you, while still being able to use
|
|
|
|
|
* become/unbecome.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
def receive(o: Any, sender: ActorRef): Unit =
|
|
|
|
|
try {
|
|
|
|
|
underlying.currentMessage =
|
|
|
|
|
Envelope(o, if (sender eq null) underlying.system.deadLetters else sender, underlying.system)
|
|
|
|
|
underlying.receiveMessage(o)
|
|
|
|
|
} finally underlying.currentMessage = null
|
2011-04-21 21:31:24 +02:00
|
|
|
|
2011-04-16 22:20:04 +02:00
|
|
|
/**
|
|
|
|
|
* Retrieve reference to the underlying actor, where the static type matches the factory used inside the
|
|
|
|
|
* constructor. Beware that this reference is discarded by the ActorRef upon restarting the actor (should this
|
|
|
|
|
* reference be linked to a supervisor). The old Actor may of course still be used in post-mortem assertions.
|
|
|
|
|
*/
|
2011-12-10 16:56:53 +01:00
|
|
|
def underlyingActor: T = {
|
|
|
|
|
// volatile mailbox read to bring in actor field
|
2017-01-13 12:06:09 +03:00
|
|
|
if (isTerminated) throw IllegalActorStateException("underlying actor is terminated")
|
2011-12-10 16:56:53 +01:00
|
|
|
underlying.actor.asInstanceOf[T] match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case null =>
|
2012-02-10 16:02:37 +01:00
|
|
|
val t = TestKitExtension(_system).DefaultTimeout
|
2011-12-17 17:18:50 -08:00
|
|
|
Await.result(this.?(InternalGetActor)(t), t.duration).asInstanceOf[T]
|
2019-02-09 15:25:39 +01:00
|
|
|
case ref => ref
|
2011-12-10 16:56:53 +01:00
|
|
|
}
|
|
|
|
|
}
|
2011-04-16 22:20:04 +02:00
|
|
|
|
2011-12-02 01:00:55 +01:00
|
|
|
/**
|
|
|
|
|
* Registers this actor to be a death monitor of the provided ActorRef
|
|
|
|
|
* This means that this actor will get a Terminated()-message when the provided actor
|
|
|
|
|
* is permanently terminated.
|
|
|
|
|
*
|
|
|
|
|
* @return the same ActorRef that is provided to it, to allow for cleaner invocations
|
|
|
|
|
*/
|
2011-12-06 09:59:52 +01:00
|
|
|
def watch(subject: ActorRef): ActorRef = underlying.watch(subject)
|
2011-12-02 01:00:55 +01:00
|
|
|
|
|
|
|
|
/**
|
2015-06-02 21:01:00 -07:00
|
|
|
* Unregisters this actor from being a death monitor of the provided ActorRef
|
2011-12-02 01:00:55 +01:00
|
|
|
* This means that this actor will not get a Terminated()-message when the provided actor
|
|
|
|
|
* is permanently terminated.
|
|
|
|
|
*
|
|
|
|
|
* @return the same ActorRef that is provided to it, to allow for cleaner invocations
|
|
|
|
|
*/
|
2011-12-06 09:59:52 +01:00
|
|
|
def unwatch(subject: ActorRef): ActorRef = underlying.unwatch(subject)
|
2011-12-02 01:00:55 +01:00
|
|
|
|
2011-11-30 23:30:56 +01:00
|
|
|
override def toString = "TestActor[" + path + "]"
|
2011-04-16 22:20:04 +02:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object TestActorRef {
|
|
|
|
|
|
2012-10-04 16:50:49 -07:00
|
|
|
private case object InternalGetActor extends AutoReceivedMessage with PossiblyHarmful
|
2012-01-31 21:38:20 +01:00
|
|
|
|
2011-11-13 20:38:14 +01:00
|
|
|
private val number = new AtomicLong
|
|
|
|
|
private[testkit] def randomName: String = {
|
|
|
|
|
val l = number.getAndIncrement()
|
|
|
|
|
"$" + akka.util.Helpers.base64(l)
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
def apply[T <: Actor: ClassTag](factory: => T)(implicit system: ActorSystem): TestActorRef[T] =
|
|
|
|
|
apply[T](Props(factory), randomName)
|
2011-04-27 15:00:41 +02:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
def apply[T <: Actor: ClassTag](factory: => T, name: String)(implicit system: ActorSystem): TestActorRef[T] =
|
|
|
|
|
apply[T](Props(factory), name)
|
2011-04-27 15:00:41 +02:00
|
|
|
|
2011-11-17 12:36:35 +01:00
|
|
|
def apply[T <: Actor](props: Props)(implicit system: ActorSystem): TestActorRef[T] = apply[T](props, randomName)
|
2011-04-27 15:00:41 +02:00
|
|
|
|
2011-11-17 12:36:35 +01:00
|
|
|
def apply[T <: Actor](props: Props, name: String)(implicit system: ActorSystem): TestActorRef[T] =
|
|
|
|
|
apply[T](props, system.asInstanceOf[ActorSystemImpl].guardian, name)
|
2011-10-18 15:39:26 +02:00
|
|
|
|
2015-07-26 09:25:42 +01:00
|
|
|
def apply[T <: Actor](props: Props, supervisor: ActorRef)(implicit system: ActorSystem): TestActorRef[T] = {
|
|
|
|
|
val sysImpl = system.asInstanceOf[ActorSystemImpl]
|
|
|
|
|
new TestActorRef(sysImpl, props, supervisor.asInstanceOf[InternalActorRef], randomName)
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(
|
|
|
|
|
implicit system: ActorSystem): TestActorRef[T] = {
|
2013-06-01 21:58:34 +02:00
|
|
|
val sysImpl = system.asInstanceOf[ActorSystemImpl]
|
|
|
|
|
new TestActorRef(sysImpl, props, supervisor.asInstanceOf[InternalActorRef], name)
|
2012-08-15 15:25:43 +02:00
|
|
|
}
|
2011-08-26 17:25:18 +02:00
|
|
|
|
2012-07-24 12:12:06 +02:00
|
|
|
def apply[T <: Actor](implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName)
|
2011-04-16 22:20:04 +02:00
|
|
|
|
2015-07-26 09:25:42 +01:00
|
|
|
private def dynamicCreateRecover[U]: PartialFunction[Throwable, U] = {
|
2019-03-11 10:38:24 +01:00
|
|
|
case exception =>
|
2019-03-13 10:56:20 +01:00
|
|
|
throw ActorInitializationException(
|
|
|
|
|
null,
|
|
|
|
|
"Could not instantiate Actor" +
|
|
|
|
|
"\nMake sure Actor is NOT defined inside a class/trait," +
|
|
|
|
|
"\nif so put it outside the class/trait, f.e. in a companion object," +
|
|
|
|
|
"\nOR try to change: 'actorOf(Props[MyActor]' to 'actorOf(Props(new MyActor)'.",
|
|
|
|
|
exception)
|
2015-07-26 09:25:42 +01:00
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
def apply[T <: Actor](name: String)(implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] =
|
|
|
|
|
apply[T](Props({
|
|
|
|
|
system
|
|
|
|
|
.asInstanceOf[ExtendedActorSystem]
|
|
|
|
|
.dynamicAccess
|
|
|
|
|
.createInstanceFor[T](t.runtimeClass, Nil)
|
|
|
|
|
.recover(dynamicCreateRecover)
|
|
|
|
|
.get
|
|
|
|
|
}), name)
|
|
|
|
|
|
|
|
|
|
def apply[T <: Actor](supervisor: ActorRef)(implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] =
|
|
|
|
|
apply[T](Props({
|
|
|
|
|
system
|
|
|
|
|
.asInstanceOf[ExtendedActorSystem]
|
|
|
|
|
.dynamicAccess
|
|
|
|
|
.createInstanceFor[T](t.runtimeClass, Nil)
|
|
|
|
|
.recover(dynamicCreateRecover)
|
|
|
|
|
.get
|
|
|
|
|
}), supervisor)
|
|
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
def apply[T <: Actor](supervisor: ActorRef, name: String)(
|
|
|
|
|
implicit t: ClassTag[T],
|
|
|
|
|
system: ActorSystem): TestActorRef[T] =
|
2019-03-11 10:38:24 +01:00
|
|
|
apply[T](
|
|
|
|
|
Props({
|
|
|
|
|
system
|
|
|
|
|
.asInstanceOf[ExtendedActorSystem]
|
|
|
|
|
.dynamicAccess
|
|
|
|
|
.createInstanceFor[T](t.runtimeClass, Nil)
|
|
|
|
|
.recover(dynamicCreateRecover)
|
|
|
|
|
.get
|
|
|
|
|
}),
|
|
|
|
|
supervisor,
|
|
|
|
|
name)
|
2015-07-26 09:25:42 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Java API: create a TestActorRef in the given system for the given props,
|
|
|
|
|
* with the given supervisor and name.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
def create[T <: Actor](system: ActorSystem, props: Props, supervisor: ActorRef, name: String): TestActorRef[T] =
|
|
|
|
|
apply(props, supervisor, name)(system)
|
2015-07-26 09:25:42 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Java API: create a TestActorRef in the given system for the given props,
|
|
|
|
|
* with the given supervisor and a random name.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
def create[T <: Actor](system: ActorSystem, props: Props, supervisor: ActorRef): TestActorRef[T] =
|
|
|
|
|
apply(props, supervisor)(system)
|
2015-07-26 09:25:42 +01:00
|
|
|
|
2012-06-25 19:30:13 +02:00
|
|
|
/**
|
2013-06-19 03:14:25 +02:00
|
|
|
* Java API: create a TestActorRef in the given system for the given props,
|
|
|
|
|
* with the given name.
|
2012-06-25 19:30:13 +02:00
|
|
|
*/
|
2012-06-29 14:42:11 +02:00
|
|
|
def create[T <: Actor](system: ActorSystem, props: Props, name: String): TestActorRef[T] = apply(props, name)(system)
|
2013-06-19 03:14:25 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Java API: create a TestActorRef in the given system for the given props,
|
|
|
|
|
* with a random name.
|
|
|
|
|
*/
|
|
|
|
|
def create[T <: Actor](system: ActorSystem, props: Props): TestActorRef[T] = apply(props)(system)
|
2011-04-21 21:31:24 +02:00
|
|
|
}
|