!act,doc,rem,tes 3823 remove deprecated Props methods

- rewrite deprecated usages to their recommended counterparts
- TestActorRef now verifies mailbox requirements
- CallingThreadMailbox now exposes proper messageQueue type for
  inspection
This commit is contained in:
Roland Kuhn 2014-01-17 09:12:44 +01:00
parent 780831469c
commit 5bcb750441
17 changed files with 68 additions and 229 deletions

View file

@ -22,7 +22,7 @@ public class ActorCreationTest {
});
assert false;
} catch(IllegalArgumentException e) {
assertEquals("cannot use non-static local Creator to create actors; make it static or top-level", e.getMessage());
assertEquals("cannot use non-static local Creator to create actors; make it static (e.g. local to a static method) or top-level", e.getMessage());
}
}

View file

@ -49,13 +49,17 @@ public class JavaAPI {
assertNotNull(ref);
}
@Test
public void mustBeAbleToCreateActorRefFromFactory() {
ActorRef ref = system.actorOf(Props.empty().withCreator(new Creator<Actor>() {
public static Props mkProps() {
return Props.create(new Creator<Actor>() {
public Actor create() {
return new JavaAPITestActor();
}
}));
});
}
@Test
public void mustBeAbleToCreateActorRefFromFactory() {
ActorRef ref = system.actorOf(mkProps());
assertNotNull(ref);
}

View file

@ -159,7 +159,7 @@ object SupervisorHierarchySpec {
val propsTemplate = Props.empty.withDispatcher("hierarchy")
(1 to kids).map { (id)
val kidSize = if (rest > 0) { rest -= 1; sizes + 1 } else sizes
val props = propsTemplate.withCreator(new Hierarchy(kidSize, breadth, listener, myLevel + 1, random))
val props = Props(new Hierarchy(kidSize, breadth, listener, myLevel + 1, random)).withDeploy(propsTemplate.deploy)
(context.watch(context.actorOf(props, id.toString)).path, kidSize)
}(collection.breakOut)
} else Map()

View file

@ -272,11 +272,6 @@ class VerifySerializabilitySpec extends AkkaSpec(SerializationTests.verifySerial
val b = system.actorOf(Props(new FooActor))
system stop b
val c = system.actorOf(Props.empty.withCreator(new UntypedActorFactory {
def create() = new FooUntypedActor
}))
system stop c
intercept[IllegalArgumentException] {
val d = system.actorOf(Props(new NonSerializableActor(system)))
}

View file

@ -310,7 +310,7 @@ private[akka] object ActorCell {
final val emptyActorRefSet: Set[ActorRef] = immutable.HashSet.empty
final val terminatedProps: Props = Props(() throw new IllegalActorStateException("This Actor has been terminated"))
final val terminatedProps: Props = Props((throw new IllegalActorStateException("This Actor has been terminated")): Actor)
final val undefinedUid = 0

View file

@ -51,7 +51,7 @@ object Props {
/**
* The default Props instance, uses the settings from the Props object starting with default*.
*/
final val default = new Props()
final val default = Props(defaultDeploy, classOf[CreatorFunctionConsumer], List(defaultCreator))
/**
* INTERNAL API
@ -87,36 +87,6 @@ object Props {
private def mkProps(classOfActor: Class[_], ctor: () Actor): Props =
Props(classOf[TypedCreatorFunctionConsumer], classOfActor, ctor)
/**
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
* using the supplied thunk.
*/
@deprecated("give class and arguments instead", "2.2")
def apply(creator: Creator[_ <: Actor]): Props = default.withCreator(creator.create)
/**
* The deprecated legacy constructor.
*/
@deprecated("use Props.withDispatcher and friends", "2.2")
def apply(
creator: () Actor = Props.defaultCreator,
dispatcher: String = Deploy.NoDispatcherGiven,
routerConfig: RouterConfig = Props.defaultRoutedProps,
deploy: Deploy = Props.defaultDeploy): Props = {
val d1 = if (dispatcher != Deploy.NoDispatcherGiven) deploy.copy(dispatcher = dispatcher) else deploy
val d2 = if (routerConfig != Props.defaultRoutedProps) d1.copy(routerConfig = routerConfig) else d1
val p = Props(classOf[CreatorFunctionConsumer], creator)
if (d2 != Props.defaultDeploy) p.withDeploy(d2) else p
}
/**
* The deprecated legacy extractor.
*/
@deprecated("use three-argument version", "2.2")
def unapply(p: Props)(dummy: Int = 0): Option[(() Actor, String, RouterConfig, Deploy)] =
Some((p.creator, p.dispatcher, p.routerConfig, p.deploy))
/**
* Scala API: create a Props given a class and its constructor arguments.
*/
@ -133,7 +103,7 @@ object Props {
*/
def create[T <: Actor](creator: Creator[T]): Props = {
if ((creator.getClass.getEnclosingClass ne null) && (creator.getClass.getModifiers & Modifier.STATIC) == 0)
throw new IllegalArgumentException("cannot use non-static local Creator to create actors; make it static or top-level")
throw new IllegalArgumentException("cannot use non-static local Creator to create actors; make it static (e.g. local to a static method) or top-level")
val ac = classOf[Actor]
val actorClass = Reflect.findMarker(creator.getClass, classOf[Creator[_]]) match {
case t: ParameterizedType
@ -199,37 +169,6 @@ final case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]
// validate producer constructor signature; throws IllegalArgumentException if invalid
producer
/**
* No-args constructor that sets all the default values.
*
* @deprecated use `Props.create(clazz, args ...)` instead
*/
@deprecated("use Props.create()", "2.2")
def this() = this(Props.defaultDeploy, classOf[CreatorFunctionConsumer], List(Props.defaultCreator))
/**
* Java API: create Props from an [[UntypedActorFactory]]
*
* @deprecated use `Props.create(clazz, args ...)` instead; this method has been
* deprecated because it encourages creating Props which contain
* non-serializable inner classes, making them also
* non-serializable
*/
@deprecated("use Props.create()", "2.2")
def this(factory: UntypedActorFactory) = this(Props.defaultDeploy, classOf[UntypedActorFactoryConsumer], List(factory))
/**
* Java API: create Props from a given [[java.lang.Class]]
*
* @deprecated use Props.create(clazz) instead; deprecated since it duplicates
* another API
*/
@deprecated("use Props.create()", "2.2")
def this(actorClass: Class[_ <: Actor]) = this(Props.defaultDeploy, actorClass, List.empty)
@deprecated("There is no use-case for this method anymore", "2.2")
def creator: () Actor = newActor
/**
* Convenience method for extracting the dispatcher information from the
* contained [[Deploy]] instance.
@ -254,36 +193,6 @@ final case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]
*/
def routerConfig: RouterConfig = deploy.routerConfig
/**
* Scala API: Returns a new Props with the specified creator set.
*
* The creator must not return the same instance multiple times.
*/
@deprecated("use Props(...).withDeploy(other.deploy)", "2.2")
def withCreator(c: Actor): Props = copy(clazz = classOf[CreatorFunctionConsumer], args = (() c) :: Nil)
/**
* Java API: Returns a new Props with the specified creator set.
*
* The creator must not return the same instance multiple times.
*
* @deprecated use `Props.create(clazz, args ...)` instead; this method has been
* deprecated because it encourages creating Props which contain
* non-serializable inner classes, making them also
* non-serializable
*/
@deprecated("use Props.create(clazz, args ...).withDeploy(other.deploy) instead", "2.2")
def withCreator(c: Creator[Actor]): Props = copy(clazz = classOf[CreatorConsumer], args = classOf[Actor] :: c :: Nil)
/**
* Returns a new Props with the specified creator set.
*
* @deprecated use Props.create(clazz) instead; deprecated since it duplicates
* another API
*/
@deprecated("use Props.create(clazz, args).withDeploy(other.deploy)", "2.2")
def withCreator(c: Class[_ <: Actor]): Props = copy(clazz = c, args = Nil)
/**
* Returns a new Props with the specified dispatcher set.
*/

View file

@ -75,7 +75,7 @@ trait TypedActorFactory {
val proxyVar = new AtomVar[R] //Chicken'n'egg-resolver
val c = props.creator //Cache this to avoid closing over the Props
val i = props.interfaces //Cache this to avoid closing over the Props
val ap = props.actorProps.withCreator(new TypedActor.TypedActor[R, T](proxyVar, c(), i))
val ap = Props(new TypedActor.TypedActor[R, T](proxyVar, c(), i)).withDeploy(props.actorProps.deploy)
typedActor.createActorRefProxy(props, proxyVar, actorFactory.actorOf(ap))
}
@ -86,7 +86,7 @@ trait TypedActorFactory {
val proxyVar = new AtomVar[R] //Chicken'n'egg-resolver
val c = props.creator //Cache this to avoid closing over the Props
val i = props.interfaces //Cache this to avoid closing over the Props
val ap = props.actorProps.withCreator(new akka.actor.TypedActor.TypedActor[R, T](proxyVar, c(), i))
val ap = Props(new akka.actor.TypedActor.TypedActor[R, T](proxyVar, c(), i)).withDeploy(props.actorProps.deploy)
typedActor.createActorRefProxy(props, proxyVar, actorFactory.actorOf(ap, name))
}
@ -619,7 +619,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
def actorProps(): Props =
if (dispatcher == Props.default.dispatcher)
Props.default.withDeploy(deploy)
else Props(dispatcher = dispatcher).withDeploy(deploy)
else Props.default.withDispatcher(dispatcher).withDeploy(deploy)
}
/**

View file

@ -58,8 +58,11 @@ private[akka] trait Dispatch { this: ActorCell ⇒
case _: ProducesMessageQueue[_] if system.mailboxes.hasRequiredType(actorClass)
val req = system.mailboxes.getRequiredType(actorClass)
if (req isInstance mbox.messageQueue) Create(None)
else Create(Some(ActorInitializationException(self,
s"Actor [$self] requires mailbox type [$req] got [${mbox.messageQueue.getClass.getName}]")))
else {
val gotType = if (mbox.messageQueue == null) "null" else mbox.messageQueue.getClass.getName
Create(Some(ActorInitializationException(self,
s"Actor [$self] requires mailbox type [$req] got [$gotType]")))
}
case _ Create(None)
}

View file

@ -11,14 +11,17 @@ import static akka.pattern.Patterns.pipe;
import static akka.pattern.Patterns.gracefulStop;
//#import-gracefulStop
import java.util.ArrayList;
import java.util.Iterator;
import java.util.concurrent.TimeUnit;
import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.ClassRule;
import org.junit.Test;
//#import-gracefulStop
import scala.concurrent.Await;
//#import-ask
@ -119,43 +122,6 @@ public class UntypedActorDocTest {
}
//#parametric-creator
@SuppressWarnings("deprecation")
@Test
public void createPropsDeprecated() {
//#creating-props-deprecated
// DEPRECATED: encourages to close over enclosing class
final Props props1 = new Props(new UntypedActorFactory() {
private static final long serialVersionUID = 1L;
@Override
public UntypedActor create() throws Exception {
return new MyUntypedActor();
}
});
// DEPRECATED: encourages to close over enclosing class
final Props props2 = new Props().withCreator(new UntypedActorFactory() {
private static final long serialVersionUID = 1L;
@Override
public UntypedActor create() throws Exception {
return new MyUntypedActor();
}
});
// these are DEPRECATED due to duplicate functionality with Props.create()
final Props props3 = new Props(MyUntypedActor.class);
final Props props4 = new Props().withCreator(MyUntypedActor.class);
//#creating-props-deprecated
new JavaTestKit(system) {
{
for (Props props : new Props[] { props1, props2, props3, props4 }) {
final ActorRef a = system.actorOf(props);
a.tell("hello", getRef());
expectMsgEquals("hello");
}
}
};
}
@Test
public void systemActorOf() {
//#system-actorOf
@ -538,8 +504,15 @@ public class UntypedActorDocTest {
* @return a Props for creating this actor, which can then be further configured
* (e.g. calling `.withDispatcher()` on it)
*/
public static Props props(int magicNumber) {
return Props.create(DemoActor.class, magicNumber);
public static Props props(final int magicNumber) {
return Props.create(new Creator<DemoActor>() {
private static final long serialVersionUID = 1L;
@Override
public DemoActor create() throws Exception {
return new DemoActor(magicNumber);
}
});
}
final int magicNumber;

View file

@ -65,52 +65,22 @@ example of a parametric factory could be:
.. includecode:: code/docs/actor/UntypedActorDocTest.java#parametric-creator
Deprecated Variants
^^^^^^^^^^^^^^^^^^^
.. note::
Up to Akka 2.1 there were also the following possibilities (which are retained
for a migration period):
.. includecode:: code/docs/actor/UntypedActorDocTest.java#import-untypedActor
.. includecode:: code/docs/actor/UntypedActorDocTest.java#creating-props-deprecated
The last two are deprecated because their functionality is available in full
through :meth:`Props.create()`.
The first two are deprecated because the resulting :class:`UntypedActorFactory`
is typically a local class which means that it implicitly carries a reference
to the enclosing class. This can easily make the resulting :class:`Props`
non-serializable, e.g. when the enclosing class is an :class:`Actor`. Akka
advocates location transparency, meaning that an application written with
actors should just work when it is deployed over multiple network nodes, and
non-serializable actor factories would break this principle. In case indirect
actor creation is needed—for example when using dependency injection—there is
the possibility to use an :class:`IndirectActorProducer` as described below.
There were two use-cases for these methods: passing constructor arguments to
the actor—which is solved by the newly introduced :meth:`Props.create()` method
above—and creating actors “on the spot” as anonymous classes. The latter should
be solved by making these actors named inner classes instead (if they are not
``static`` then the enclosing instances ``this`` reference needs to be passed
as the first argument).
.. warning::
Declaring one actor within another is very dangerous and breaks actor
encapsulation unless the nested actor is a static inner class. Never pass an
actors ``this`` reference into :class:`Props`!
In order for mailbox requirements—like using a deque-based mailbox for actors
using the stash—to be picked up, the actor type needs to be known before
creating it, which is what the :class:`Creator` type argument allows.
Therefore make sure to use the specific type for your actors wherever
possible.
Recommended Practices
^^^^^^^^^^^^^^^^^^^^^
It is a good idea to provide static factory methods on the
:class:`UntypedActor` which help keeping the creation of suitable
:class:`Props` as close to the actor definition as possible, thus containing
the gap in type-safety introduced by reflective instantiation within a single
class instead of spreading it out across a whole code-base. This helps
especially when refactoring the actors constructor signature at a later point,
where compiler checks will allow this modification to be done with greater
confidence than without.
:class:`Props` as close to the actor definition as possible. This also allows
usage of the :class:`Creator`-based methods which statically verify that the
used constructor actually exists instead relying only on a runtime check.
.. includecode:: code/docs/actor/UntypedActorDocTest.java#props-factory

View file

@ -132,6 +132,7 @@ The following, previously deprecated, features have been removed:
* `event-handlers renamed to loggers <http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#event-handlers_renamed_to_loggers>`_
* `API changes to FSM and TestFSMRef <http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#API_changes_to_FSM_and_TestFSMRef>`_
* DefaultScheduler superseded by LightArrayRevolverScheduler
* all previously deprecated construction and deconstruction methods for Props
publishCurrentClusterState is Deprecated
========================================

View file

@ -81,31 +81,19 @@ verified during construction of the :class:`Props` object, resulting in an
:class:`IllegalArgumentEception` if no or multiple matching constructors are
found.
Deprecated Variants
^^^^^^^^^^^^^^^^^^^
Up to Akka 2.1 there were also the following possibilities (which are retained
for a migration period):
Dangerous Variants
^^^^^^^^^^^^^^^^^^
.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props-deprecated
The first one is deprecated because the case class structure changed between
Akka 2.1 and 2.2.
The two variants in the middle are deprecated because :class:`Props` are
primarily concerned with actor creation and thus the “creator” part should be
explicitly set when creating an instance. In case you want to deploy one actor
in the same was as another, simply use
``Props(...).withDeploy(otherProps.deploy)``.
The last one is not technically deprecated, but it is not recommended to be
used within another actor because it encourages to close over the enclosing
scope, resulting in non-serializable :class:`Props` and possibly race
conditions (breaking the actor encapsulation). We will provide a macro-based
solution in a future release which allows similar syntax without the headaches,
at which point this variant will be properly deprecated. On the other hand
using this variant in a :class:`Props` factory in the actors companion object as
documented under “Recommended Practices” below is completely fine.
This method is not recommended to be used within another actor because it
encourages to close over the enclosing scope, resulting in non-serializable
:class:`Props` and possibly race conditions (breaking the actor encapsulation).
We will provide a macro-based solution in a future release which allows similar
syntax without the headaches, at which point this variant will be properly
deprecated. On the other hand using this variant in a :class:`Props` factory in
the actors companion object as documented under “Recommended Practices” below
is completely fine.
There were two use-cases for these methods: passing constructor arguments to
the actor—which is solved by the newly introduced

View file

@ -251,17 +251,6 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
//#creating-props
//#creating-props-deprecated
// DEPRECATED: old case class signature
val props4 = Props(
creator = { () => new MyActor },
dispatcher = "my-dispatcher")
// DEPRECATED due to duplicate functionality with Props.apply()
val props5 = props1.withCreator(new MyActor)
// DEPRECATED due to duplicate functionality with Props.apply()
val props6 = props1.withCreator(classOf[MyActor])
// NOT RECOMMENDED within another actor:
// encourages to close over enclosing class
val props7 = Props(new MyActor)

View file

@ -48,7 +48,7 @@ class DaemonMsgCreateSerializerSpec extends AkkaSpec {
"serialize and de-serialize DaemonMsgCreate with function creator" in {
verifySerialization {
DaemonMsgCreate(
props = Props.empty.withCreator(new MyActor),
props = Props(new MyActor),
deploy = Deploy(),
path = "foo",
supervisor = supervisor)

View file

@ -320,6 +320,12 @@ class CallingThreadMailbox(_receiver: akka.actor.Cell, val mailboxType: MailboxT
}
}
/**
* This is only a marker to be put in the messageQueues stead to make error
* messages pertaining to violated mailbox type requirements less cryptic.
*/
override val messageQueue: MessageQueue = q.get
override def enqueue(receiver: ActorRef, msg: Envelope): Unit = q.get.enqueue(receiver, msg)
override def dequeue(): Envelope = throw new UnsupportedOperationException("CallingThreadMailbox cannot dequeue normally")
override def hasMessages: Boolean = q.get.hasMessages

View file

@ -127,9 +127,9 @@ object TestActorRef {
"$" + akka.util.Helpers.base64(l)
}
def apply[T <: Actor](factory: T)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props.empty.withCreator(factory), randomName)
def apply[T <: Actor: ClassTag](factory: T)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props(factory), randomName)
def apply[T <: Actor](factory: T, name: String)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props.empty.withCreator(factory), name)
def apply[T <: Actor: ClassTag](factory: T, name: String)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props(factory), name)
def apply[T <: Actor](props: Props)(implicit system: ActorSystem): TestActorRef[T] = apply[T](props, randomName)

View file

@ -10,6 +10,7 @@ import akka.dispatch.DispatcherPrerequisites
import scala.concurrent.duration.FiniteDuration
import akka.dispatch.MessageDispatcher
import akka.dispatch.MailboxType
import scala.reflect.ClassTag
/**
* This is a specialised form of the TestActorRef with support for querying and
@ -89,13 +90,13 @@ class TestFSMRef[S, D, T <: Actor](
object TestFSMRef {
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
def apply[S, D, T <: Actor: ClassTag](factory: T)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
val impl = system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559
new TestFSMRef(impl, Props(creator = () factory), impl.guardian.asInstanceOf[InternalActorRef], TestActorRef.randomName)
new TestFSMRef(impl, Props(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] = {
def apply[S, D, T <: Actor: ClassTag](factory: T, name: String)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
val impl = system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559
new TestFSMRef(impl, Props(creator = () factory), impl.guardian.asInstanceOf[InternalActorRef], name)
new TestFSMRef(impl, Props(factory), impl.guardian.asInstanceOf[InternalActorRef], name)
}
}