pekko/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala

144 lines
5.6 KiB
Scala
Raw Normal View History

2011-04-27 15:00:41 +02:00
/**
2012-01-19 18:21:06 +01:00
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
2011-04-27 15:00:41 +02:00
*/
package akka.testkit
import akka.actor._
Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750 - PropertyMaster is the only place in Akka which calls ClassLoader.getClass (apart from kernel, which might be special) - all PropertyMaster methods (there are only three) take a ClassManifest of what is to be constructed, and they verify that the obtained object is actually compatible with the required type Other stuff: - noticed that I had forgotten to change to ExtendedActorSystem when constructing Extensions by ExtensionKey (damn you, reflection!) - moved Serializer.currentSystem into JavaSerializer, because that’s the only one needing it (it’s only used in readResolve() methods) - Serializers are constructed now with one-arg constructor taking ExtendedActorSystem (if that exists, otherwise no-arg as before), to allow JavaSerializer to do its magic; possibly necessary for others as well - Removed all Option[ClassLoader] signatures - made it so that the ActorSystem will try context class loader, then the class loader which loaded the class actually calling into ActorSystem.apply, then the loader which loaded ActorSystemImpl - for the second of the above I added a (reflectively accessed hopefully safe) facility for getting caller Class[_] objects by using sun.reflect.Reflection; this is optional an defaults to None, e.g. on Android, which means that getting the caller’s classloader is done on a best effort basis (there’s nothing we can do because a StackTrace does not contain actual Class[_] objects). - refactored DurableMailbox to contain the owner val and use that instead of declaring that in all subclasses
2012-02-09 11:56:43 +01:00
import akka.util.Duration
import java.util.concurrent.atomic.AtomicLong
import scala.collection.immutable.Stack
import akka.dispatch._
import akka.pattern.ask
2011-04-27 15:00:41 +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.
*
* @author Roland Kuhn
* @since 1.1
*/
class TestActorRef[T <: Actor](
_system: ActorSystemImpl,
_prerequisites: DispatcherPrerequisites,
_props: Props,
_supervisor: InternalActorRef,
name: String)
extends LocalActorRef(
_system,
_props.withDispatcher(
if (_props.dispatcher == Dispatchers.DefaultDispatcherId) CallingThreadDispatcher.Id
else _props.dispatcher),
_supervisor,
_supervisor.path / name,
false) {
import TestActorRef.InternalGetActor
override def newActorCell(
system: ActorSystemImpl,
ref: InternalActorRef,
props: Props,
supervisor: InternalActorRef,
receiveTimeout: Option[Duration]): ActorCell =
new ActorCell(system, ref, props, supervisor, receiveTimeout) {
override def autoReceiveMessage(msg: Envelope) {
msg.message match {
case InternalGetActor sender ! actor
case _ super.autoReceiveMessage(msg)
}
}
}
/**
* Directly inject messages into actor receive behavior. Any exceptions
* thrown will be available to you, while still being able to use
* become/unbecome.
*/
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.
*/
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
/**
* 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.
*/
def underlyingActor: T = {
// volatile mailbox read to bring in actor field
if (isTerminated) throw new IllegalActorStateException("underlying actor is terminated")
underlying.actor.asInstanceOf[T] match {
case null
val t = TestKitExtension(_system).DefaultTimeout
2011-12-17 17:18:50 -08:00
Await.result(this.?(InternalGetActor)(t), t.duration).asInstanceOf[T]
case ref ref
}
}
/**
* 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
*/
def watch(subject: ActorRef): ActorRef = underlying.watch(subject)
/**
* Deregisters this actor from being a death monitor of the provided ActorRef
* 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
*/
def unwatch(subject: ActorRef): ActorRef = underlying.unwatch(subject)
override def toString = "TestActor[" + path + "]"
}
object TestActorRef {
private case object InternalGetActor extends AutoReceivedMessage
private val number = new AtomicLong
private[testkit] def randomName: String = {
val l = number.getAndIncrement()
"$" + akka.util.Helpers.base64(l)
}
def apply[T <: Actor](factory: T)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props(factory), randomName)
2011-04-27 15:00:41 +02:00
def apply[T <: Actor](factory: T, name: String)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props(factory), name)
2011-04-27 15:00:41 +02:00
def apply[T <: Actor](props: Props)(implicit system: ActorSystem): TestActorRef[T] = apply[T](props, randomName)
2011-04-27 15:00:41 +02:00
def apply[T <: Actor](props: Props, name: String)(implicit system: ActorSystem): TestActorRef[T] =
apply[T](props, system.asInstanceOf[ActorSystemImpl].guardian, name)
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit system: ActorSystem): TestActorRef[T] =
new TestActorRef(system.asInstanceOf[ActorSystemImpl], system.dispatchers.prerequisites, props, supervisor.asInstanceOf[InternalActorRef], name)
2011-08-26 17:25:18 +02:00
def apply[T <: Actor](implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName)
def apply[T <: Actor](name: String)(implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](Props({
system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](m.erasure, Seq()) match {
case Right(value) value
case Left(exception) throw new 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)
2011-04-29 17:15:00 +02:00
}
}), name)
}