diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 766da4161b..098d9f5828 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -73,9 +73,9 @@ private[akka] object ActorCell { //Make sure that they are not read/written outside of a message processing (systemInvoke/invoke) private[akka] class ActorCell( val system: ActorSystemImpl, - val self: ActorRef with ScalaActorRef, + val self: InternalActorRef, val props: Props, - val parent: ActorRef, + val parent: InternalActorRef, /*no member*/ _receiveTimeout: Option[Long], var hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index b09cefdc9d..c15663d024 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -48,7 +48,7 @@ import akka.event.DeathWatch * @author Jonas Bonér */ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable { - scalaRef: ScalaActorRef with RefInternals ⇒ + scalaRef: InternalActorRef ⇒ // Only mutable for RemoteServer in order to maintain identity across nodes /** @@ -121,6 +121,53 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable override def toString = "Actor[%s]".format(path) } +/** + * This trait represents the Scala Actor API + * There are implicit conversions in ../actor/Implicits.scala + * from ActorRef -> ScalaActorRef and back + */ +trait ScalaActorRef { ref: ActorRef ⇒ + + /** + * Sends a one-way asynchronous message. E.g. fire-and-forget semantics. + *
+ * + * If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument. + * + * + * This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable, + * if invoked from within an Actor. If not then no sender is available. + *+ * actor ! message + *+ * + */ + def !(message: Any)(implicit sender: ActorRef = null): Unit + + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + */ + def ?(message: Any)(implicit timeout: Timeout): Future[Any] + + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The implicit parameter with the default value is just there to disambiguate it from the version that takes the + * implicit timeout + */ + def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) +} + +/** + * Internal trait for assembling all the functionality needed internally on + * ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE! + */ +private[akka] trait InternalActorRef extends ActorRef with ScalaActorRef { + def resume(): Unit + def suspend(): Unit + def restart(cause: Throwable): Unit + def sendSystemMessage(message: SystemMessage): Unit +} + /** * Local (serializable) ActorRef that is used when referencing the Actor on its "home" node. * @@ -129,12 +176,12 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable class LocalActorRef private[akka] ( system: ActorSystemImpl, _props: Props, - _supervisor: ActorRef, + _supervisor: InternalActorRef, val path: ActorPath, val systemService: Boolean = false, _receiveTimeout: Option[Long] = None, _hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap) - extends ActorRef with ScalaActorRef with RefInternals { + extends InternalActorRef { /* * actorCell.start() publishes actorCell & this to the dispatcher, which @@ -191,62 +238,18 @@ class LocalActorRef private[akka] ( instance } - protected[akka] def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) } + def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) } def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender) def ?(message: Any)(implicit timeout: Timeout): Future[Any] = actorCell.provider.ask(message, this, timeout) - protected[akka] override def restart(cause: Throwable): Unit = actorCell.restart(cause) + def restart(cause: Throwable): Unit = actorCell.restart(cause) @throws(classOf[java.io.ObjectStreamException]) private def writeReplace(): AnyRef = SerializedActorRef(path.toString) } -/** - * This trait represents the Scala Actor API - * There are implicit conversions in ../actor/Implicits.scala - * from ActorRef -> ScalaActorRef and back - */ -trait ScalaActorRef { ref: ActorRef ⇒ - - protected[akka] def sendSystemMessage(message: SystemMessage): Unit - - /** - * Sends a one-way asynchronous message. E.g. fire-and-forget semantics. - * - * - * If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument. - * - * - * This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable, - * if invoked from within an Actor. If not then no sender is available. - *
- * actor ! message - *- * - */ - def !(message: Any)(implicit sender: ActorRef = null): Unit - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The implicit parameter with the default value is just there to disambiguate it from the version that takes the - * implicit timeout - */ - def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) -} - -private[akka] trait RefInternals { - def resume(): Unit - def suspend(): Unit - protected[akka] def restart(cause: Throwable): Unit -} - /** * Memento pattern for serializing ActorRefs transparently */ @@ -266,7 +269,7 @@ case class SerializedActorRef(path: String) { /** * Trait for ActorRef implementations where all methods contain default stubs. */ -trait MinimalActorRef extends ActorRef with ScalaActorRef with RefInternals { +trait MinimalActorRef extends InternalActorRef { //FIXME REMOVE THIS, ticket #1416 //FIXME REMOVE THIS, ticket #1415 @@ -282,8 +285,8 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef with RefInternals { def ?(message: Any)(implicit timeout: Timeout): Future[Any] = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName)) - protected[akka] def sendSystemMessage(message: SystemMessage): Unit = () - protected[akka] def restart(cause: Throwable): Unit = () + def sendSystemMessage(message: SystemMessage): Unit = () + def restart(cause: Throwable): Unit = () } object MinimalActorRef { @@ -355,7 +358,7 @@ class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deathWatch: D case other ⇒ result.completeWithResult(other) } - protected[akka] override def sendSystemMessage(message: SystemMessage): Unit = message match { + override def sendSystemMessage(message: SystemMessage): Unit = message match { case _: Terminate ⇒ stop() case _ ⇒ } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 0de2de5b05..b2271c85c7 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -30,12 +30,12 @@ trait ActorRefProvider { /** * Reference to the supervisor used for all top-level user actors. */ - def guardian: ActorRef + def guardian: InternalActorRef /** * Reference to the supervisor used for all top-level system actors. */ - def systemGuardian: ActorRef + def systemGuardian: InternalActorRef /** * Reference to the death watch service. @@ -74,26 +74,20 @@ trait ActorRefProvider { * in case of remote supervision). If systemService is true, deployment is * bypassed (local-only). */ - def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean = false): ActorRef + def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, name: String, systemService: Boolean = false): InternalActorRef /** - * <<<<<<< HEAD * Create actor reference for a specified local or remote path. If no such * actor exists, it will be (equivalent to) a dead letter reference. - * ======= - * Create an Actor with the given full path below the given supervisor. - * - * FIXME: Remove! this is dangerous!? - * >>>>>>> master */ - def actorFor(path: ActorPath): ActorRef + def actorFor(path: ActorPath): InternalActorRef /** * Create actor reference for a specified local or remote path, which will * be parsed using java.net.URI. If no such actor exists, it will be * (equivalent to) a dead letter reference. */ - def actorFor(s: String): ActorRef + def actorFor(s: String): InternalActorRef /** * Create actor reference for the specified child path starting at the root @@ -101,7 +95,7 @@ trait ActorRefProvider { * i.e. it cannot be used to obtain a reference to an actor which is not * physically or logically attached to this actor system. */ - def actorFor(p: Iterable[String]): ActorRef + def actorFor(p: Iterable[String]): InternalActorRef private[akka] def createDeathWatch(): DeathWatch @@ -131,7 +125,7 @@ trait ActorRefFactory { /** * Father of all children created by this interface. */ - protected def guardian: ActorRef + protected def guardian: InternalActorRef protected def randomName(): String @@ -175,7 +169,7 @@ trait ActorRefFactory { def actorOf(creator: UntypedActorFactory, name: String): ActorRef = actorOf(Props(() ⇒ creator.create()), name) - def actorFor(path: ActorPath) = provider.actorFor(path) + def actorFor(path: ActorPath): ActorRef = provider.actorFor(path) def actorFor(path: String): ActorRef = provider.actorFor(path) @@ -192,7 +186,7 @@ class LocalActorRefProvider( val settings: ActorSystem.Settings, val eventStream: EventStream, val scheduler: Scheduler, - val deadLetters: ActorRef) extends ActorRefProvider { + val deadLetters: InternalActorRef) extends ActorRefProvider { val rootPath: ActorPath = new RootActorPath(LocalAddress(_systemName)) @@ -219,7 +213,7 @@ class LocalActorRefProvider( * Top-level anchor for the supervision hierarchy of this actor system. Will * receive only Supervise/ChildTerminated system messages or Failure message. */ - private[akka] val theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = new MinimalActorRef { + private[akka] val theOneWhoWalksTheBubblesOfSpaceTime: InternalActorRef = new MinimalActorRef { val stopped = new Switch(false) @volatile @@ -242,7 +236,7 @@ class LocalActorRefProvider( case _ ⇒ log.error(this + " received unexpected message " + message) }) - protected[akka] override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff { + override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff { message match { case Supervise(child) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead case _ ⇒ log.error(this + " received unexpected system message " + message) @@ -287,9 +281,9 @@ class LocalActorRefProvider( def dispatcher: MessageDispatcher = system.dispatcher lazy val terminationFuture: DefaultPromise[Unit] = new DefaultPromise[Unit](Timeout.never)(dispatcher) - lazy val rootGuardian: ActorRef = new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) - lazy val guardian: ActorRef = actorOf(system, guardianProps, rootGuardian, "user", true) - lazy val systemGuardian: ActorRef = actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, "system", true) + lazy val rootGuardian: InternalActorRef = new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) + lazy val guardian: InternalActorRef = actorOf(system, guardianProps, rootGuardian, "user", true) + lazy val systemGuardian: InternalActorRef = actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, "system", true) val deathWatch = createDeathWatch() @@ -300,18 +294,18 @@ class LocalActorRefProvider( deathWatch.subscribe(rootGuardian, systemGuardian) } - def actorFor(path: String): ActorRef = path match { + def actorFor(path: String): InternalActorRef = path match { case LocalActorPath(address, elems) if address == rootPath.address ⇒ findInTree(rootGuardian.asInstanceOf[LocalActorRef], elems) case _ ⇒ deadLetters } - def actorFor(path: ActorPath): ActorRef = findInTree(rootGuardian.asInstanceOf[LocalActorRef], path.pathElements) + def actorFor(path: ActorPath): InternalActorRef = findInTree(rootGuardian.asInstanceOf[LocalActorRef], path.pathElements) - def actorFor(path: Iterable[String]): ActorRef = findInTree(rootGuardian.asInstanceOf[LocalActorRef], path) + def actorFor(path: Iterable[String]): InternalActorRef = findInTree(rootGuardian.asInstanceOf[LocalActorRef], path) @tailrec - private def findInTree(start: LocalActorRef, path: Iterable[String]): ActorRef = { + private def findInTree(start: LocalActorRef, path: Iterable[String]): InternalActorRef = { if (path.isEmpty) start else start.underlying.getChild(path.head) match { case null ⇒ deadLetters @@ -320,7 +314,7 @@ class LocalActorRefProvider( } } - def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = { + def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, name: String, systemService: Boolean): InternalActorRef = { val path = supervisor.path / name (if (systemService) None else deployer.lookupDeployment(path.toString)) match { @@ -359,7 +353,7 @@ class LocalActorRefProvider( /** * Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration. */ - def actorOf(system: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = { + def actorOf(system: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): InternalActorRef = { // FIXME: this needs to take supervision into account! //FIXME clustering should be implemented by cluster actor ref provider diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 6d9b0b7da7..35b100913c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -355,7 +355,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor classOf[Settings] -> settings, classOf[EventStream] -> eventStream, classOf[Scheduler] -> scheduler, - classOf[ActorRef] -> deadLetters) + classOf[InternalActorRef] -> deadLetters) val types: Array[Class[_]] = arguments map (_._1) toArray val values: Array[AnyRef] = arguments map (_._2) toArray @@ -382,8 +382,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher def terminationFuture: Future[Unit] = provider.terminationFuture - def guardian: ActorRef = provider.guardian - def systemGuardian: ActorRef = provider.systemGuardian + def guardian: InternalActorRef = provider.guardian + def systemGuardian: InternalActorRef = provider.systemGuardian def deathWatch: DeathWatch = provider.deathWatch def nodename: String = provider.nodename def clustername: String = provider.clustername diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index b079550998..e239f50de1 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -122,12 +122,12 @@ abstract class FaultHandlingStrategy { def handleSupervisorFailing(supervisor: ActorRef, children: Iterable[ActorRef]): Unit = { if (children.nonEmpty) - children.foreach(_.asInstanceOf[RefInternals].suspend()) + children.foreach(_.asInstanceOf[InternalActorRef].suspend()) } def handleSupervisorRestarted(cause: Throwable, supervisor: ActorRef, children: Iterable[ActorRef]): Unit = { if (children.nonEmpty) - children.foreach(_.asInstanceOf[RefInternals].restart(cause)) + children.foreach(_.asInstanceOf[InternalActorRef].restart(cause)) } /** @@ -136,7 +136,7 @@ abstract class FaultHandlingStrategy { def handleFailure(child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = { val action = if (decider.isDefinedAt(cause)) decider(cause) else Escalate action match { - case Resume ⇒ child.asInstanceOf[RefInternals].resume(); true + case Resume ⇒ child.asInstanceOf[InternalActorRef].resume(); true case Restart ⇒ processFailure(true, child, cause, stats, children); true case Stop ⇒ processFailure(false, child, cause, stats, children); true case Escalate ⇒ false @@ -194,7 +194,7 @@ case class AllForOneStrategy(decider: FaultHandlingStrategy.Decider, def processFailure(restart: Boolean, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Unit = { if (children.nonEmpty) { if (restart && children.forall(_.requestRestartPermission(retriesWindow))) - children.foreach(_.child.asInstanceOf[RefInternals].restart(cause)) + children.foreach(_.child.asInstanceOf[InternalActorRef].restart(cause)) else children.foreach(_.child.stop()) } @@ -247,7 +247,7 @@ case class OneForOneStrategy(decider: FaultHandlingStrategy.Decider, def processFailure(restart: Boolean, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Unit = { if (restart && stats.requestRestartPermission(retriesWindow)) - child.asInstanceOf[RefInternals].restart(cause) + child.asInstanceOf[InternalActorRef].restart(cause) else child.stop() //TODO optimization to drop child here already? } diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 71ca70ec99..acd40ac3a1 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -56,7 +56,7 @@ class Remote(val system: ActorSystemImpl, val nodename: String) { private[remote] lazy val remoteDaemon = system.provider.actorOf(system, Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)), - remoteDaemonSupervisor, + remoteDaemonSupervisor.asInstanceOf[InternalActorRef], remoteDaemonServiceName, systemService = true) @@ -141,7 +141,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { message.getActorPath match { case RemoteActorPath(addr, elems) if addr == remoteAddress && elems.size > 0 ⇒ val name = elems.last - system.actorFor(elems.dropRight(1)) match { + systemImpl.provider.actorFor(elems.dropRight(1)) match { case x if x eq system.deadLetters ⇒ log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message) case parent ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index cf55c8edb1..84febbb202 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -34,7 +34,7 @@ class RemoteActorRefProvider( val settings: ActorSystem.Settings, val eventStream: EventStream, val scheduler: Scheduler, - _deadLetters: ActorRef) extends ActorRefProvider { + _deadLetters: InternalActorRef) extends ActorRefProvider { val log = Logging(eventStream, "RemoteActorRefProvider") @@ -79,7 +79,7 @@ class RemoteActorRefProvider( def dispatcher = local.dispatcher def defaultTimeout = settings.ActorTimeout - def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = + def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, name: String, systemService: Boolean): InternalActorRef = if (systemService) local.actorOf(system, props, supervisor, name, systemService) else { val path = supervisor.path / name @@ -87,7 +87,7 @@ class RemoteActorRefProvider( actors.putIfAbsent(path.toString, newFuture) match { // we won the race -- create the actor and resolve the future case null ⇒ - val actor: ActorRef = try { + val actor: InternalActorRef = try { deployer.lookupDeploymentFor(path.toString) match { case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.RemoteScope(remoteAddresses))) ⇒ @@ -166,8 +166,8 @@ class RemoteActorRefProvider( newFuture completeWithResult actor actors.replace(path.toString, newFuture, actor) actor - case actor: ActorRef ⇒ actor - case future: Future[_] ⇒ future.get.asInstanceOf[ActorRef] + case actor: InternalActorRef ⇒ actor + case future: Future[_] ⇒ future.get.asInstanceOf[InternalActorRef] } } @@ -175,14 +175,14 @@ class RemoteActorRefProvider( * Copied from LocalActorRefProvider... */ // FIXME: implement supervision, ticket #1408 - def actorOf(system: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = { + def actorOf(system: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): InternalActorRef = { if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router") new RoutedActorRef(system, props, supervisor, name) } - def actorFor(path: ActorPath): ActorRef = local.actorFor(path) - def actorFor(path: String): ActorRef = local.actorFor(path) - def actorFor(path: Iterable[String]): ActorRef = local.actorFor(path) + def actorFor(path: ActorPath): InternalActorRef = local.actorFor(path) + def actorFor(path: String): InternalActorRef = local.actorFor(path) + def actorFor(path: Iterable[String]): InternalActorRef = local.actorFor(path) // TODO remove me val optimizeLocal = new AtomicBoolean(true) @@ -265,7 +265,7 @@ private[akka] case class RemoteActorRef private[akka] ( remoteAddress: RemoteAddress, path: ActorPath, loader: Option[ClassLoader]) - extends ActorRef with ScalaActorRef with RefInternals { + extends InternalActorRef { @volatile private var running: Boolean = true @@ -276,7 +276,7 @@ private[akka] case class RemoteActorRef private[akka] ( def isTerminated: Boolean = !running - protected[akka] def sendSystemMessage(message: SystemMessage): Unit = throw new UnsupportedOperationException("Not supported for RemoteActorRef") + def sendSystemMessage(message: SystemMessage): Unit = throw new UnsupportedOperationException("Not supported for RemoteActorRef") override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), remoteAddress, this, loader) @@ -298,5 +298,5 @@ private[akka] case class RemoteActorRef private[akka] ( @throws(classOf[java.io.ObjectStreamException]) private def writeReplace(): AnyRef = SerializedActorRef(path.toString) - protected[akka] def restart(cause: Throwable): Unit = () + def restart(cause: Throwable): Unit = () } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index d974236824..ce1454b3b1 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -25,7 +25,7 @@ class TestActorRef[T <: Actor]( _system: ActorSystemImpl, _prerequisites: DispatcherPrerequisites, _props: Props, - _supervisor: ActorRef, + _supervisor: InternalActorRef, name: String) extends LocalActorRef(_system, _props.withDispatcher(new CallingThreadDispatcher(_prerequisites)), _supervisor, _supervisor.path / name, false) { /** @@ -86,7 +86,7 @@ object TestActorRef { 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.dispatcherFactory.prerequisites, props, supervisor, name) + new TestActorRef(system.asInstanceOf[ActorSystemImpl], system.dispatcherFactory.prerequisites, props, supervisor.asInstanceOf[InternalActorRef], name) def apply[T <: Actor](implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index 36143965c3..17c8a45ade 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -40,7 +40,7 @@ class TestFSMRef[S, D, T <: Actor]( system: ActorSystemImpl, _prerequisites: DispatcherPrerequisites, props: Props, - supervisor: ActorRef, + supervisor: InternalActorRef, name: String)(implicit ev: T <:< FSM[S, D]) extends TestActorRef(system, _prerequisites, props, supervisor, name) { @@ -89,11 +89,11 @@ object TestFSMRef { def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { val impl = system.asInstanceOf[ActorSystemImpl] - new TestFSMRef(impl, system.dispatcherFactory.prerequisites, Props(creator = () ⇒ factory), impl.guardian, TestActorRef.randomName) + new TestFSMRef(impl, system.dispatcherFactory.prerequisites, Props(creator = () ⇒ factory), impl.guardian.asInstanceOf[InternalActorRef], TestActorRef.randomName) } def apply[S, D, T <: Actor](factory: ⇒ T, name: String)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { val impl = system.asInstanceOf[ActorSystemImpl] - new TestFSMRef(impl, system.dispatcherFactory.prerequisites, Props(creator = () ⇒ factory), impl.guardian, name) + new TestFSMRef(impl, system.dispatcherFactory.prerequisites, Props(creator = () ⇒ factory), impl.guardian.asInstanceOf[InternalActorRef], name) } } diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 12096a61b1..afbca071be 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -173,8 +173,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { counter = 2 val boss = TestActorRef(Props(new TActor { - val impl = system.asInstanceOf[ActorSystemImpl] - val ref = new TestActorRef(impl, impl.dispatcherFactory.prerequisites, Props(new TActor { + val ref = TestActorRef(Props(new TActor { def receiveT = { case _ ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 }