Merge pull request #1336 from akka/wip-3081-PropsClosures-∂π
deprecate closure-taking Props factories, see #3081
This commit is contained in:
commit
57d71b0b44
77 changed files with 2234 additions and 1592 deletions
|
|
@ -408,7 +408,10 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
}
|
}
|
||||||
|
|
||||||
"continue to process messages when a thread gets interrupted and throws an exception" in {
|
"continue to process messages when a thread gets interrupted and throws an exception" in {
|
||||||
filterEvents(EventFilter[InterruptedException](), EventFilter[akka.event.Logging.LoggerException]()) {
|
filterEvents(
|
||||||
|
EventFilter[InterruptedException](),
|
||||||
|
EventFilter[ActorInterruptedException](),
|
||||||
|
EventFilter[akka.event.Logging.LoggerException]()) {
|
||||||
implicit val dispatcher = interceptedDispatcher()
|
implicit val dispatcher = interceptedDispatcher()
|
||||||
val a = newTestActor(dispatcher.id)
|
val a = newTestActor(dispatcher.id)
|
||||||
val f1 = a ? Reply("foo")
|
val f1 = a ? Reply("foo")
|
||||||
|
|
@ -435,6 +438,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
assert(Await.result(f6, remaining) === "bar2")
|
assert(Await.result(f6, remaining) === "bar2")
|
||||||
assert(intercept[ActorInterruptedException](Await.result(f5, remaining)).getCause.getMessage === "Ping!")
|
assert(intercept[ActorInterruptedException](Await.result(f5, remaining)).getCause.getMessage === "Ping!")
|
||||||
c.cancel()
|
c.cancel()
|
||||||
|
Thread.sleep(300) // give the EventFilters a chance of catching all messages
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -338,8 +338,11 @@ object Actor {
|
||||||
/**
|
/**
|
||||||
* Type alias representing a Receive-expression for Akka Actors.
|
* Type alias representing a Receive-expression for Akka Actors.
|
||||||
*/
|
*/
|
||||||
|
//#receive
|
||||||
type Receive = PartialFunction[Any, Unit]
|
type Receive = PartialFunction[Any, Unit]
|
||||||
|
|
||||||
|
//#receive
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* emptyBehavior is a Receive-expression that matches no messages at all, ever.
|
* emptyBehavior is a Receive-expression that matches no messages at all, ever.
|
||||||
*/
|
*/
|
||||||
|
|
@ -463,7 +466,9 @@ trait Actor {
|
||||||
* This defines the initial actor behavior, it must return a partial function
|
* This defines the initial actor behavior, it must return a partial function
|
||||||
* with the actor logic.
|
* with the actor logic.
|
||||||
*/
|
*/
|
||||||
def receive: Receive
|
//#receive
|
||||||
|
def receive: Actor.Receive
|
||||||
|
//#receive
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* User overridable definition the strategy to use for supervising
|
* User overridable definition the strategy to use for supervising
|
||||||
|
|
@ -478,8 +483,11 @@ trait Actor {
|
||||||
* Actors are automatically started asynchronously when created.
|
* Actors are automatically started asynchronously when created.
|
||||||
* Empty default implementation.
|
* Empty default implementation.
|
||||||
*/
|
*/
|
||||||
@throws(classOf[Exception])
|
@throws(classOf[Exception]) // when changing this you MUST also change UntypedActorDocTestBase
|
||||||
def preStart() {}
|
//#lifecycle-hooks
|
||||||
|
def preStart(): Unit = ()
|
||||||
|
|
||||||
|
//#lifecycle-hooks
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* User overridable callback.
|
* User overridable callback.
|
||||||
|
|
@ -487,8 +495,11 @@ trait Actor {
|
||||||
* Is called asynchronously after 'actor.stop()' is invoked.
|
* Is called asynchronously after 'actor.stop()' is invoked.
|
||||||
* Empty default implementation.
|
* Empty default implementation.
|
||||||
*/
|
*/
|
||||||
@throws(classOf[Exception])
|
@throws(classOf[Exception]) // when changing this you MUST also change UntypedActorDocTestBase
|
||||||
def postStop() {}
|
//#lifecycle-hooks
|
||||||
|
def postStop(): Unit = ()
|
||||||
|
|
||||||
|
//#lifecycle-hooks
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* User overridable callback: '''By default it disposes of all children and then calls `postStop()`.'''
|
* User overridable callback: '''By default it disposes of all children and then calls `postStop()`.'''
|
||||||
|
|
@ -498,8 +509,9 @@ trait Actor {
|
||||||
* Is called on a crashed Actor right BEFORE it is restarted to allow clean
|
* Is called on a crashed Actor right BEFORE it is restarted to allow clean
|
||||||
* up of resources before Actor is terminated.
|
* up of resources before Actor is terminated.
|
||||||
*/
|
*/
|
||||||
@throws(classOf[Exception])
|
@throws(classOf[Exception]) // when changing this you MUST also change UntypedActorDocTestBase
|
||||||
def preRestart(reason: Throwable, message: Option[Any]) {
|
//#lifecycle-hooks
|
||||||
|
def preRestart(reason: Throwable, message: Option[Any]): Unit = {
|
||||||
context.children foreach { child ⇒
|
context.children foreach { child ⇒
|
||||||
context.unwatch(child)
|
context.unwatch(child)
|
||||||
context.stop(child)
|
context.stop(child)
|
||||||
|
|
@ -507,14 +519,20 @@ trait Actor {
|
||||||
postStop()
|
postStop()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
//#lifecycle-hooks
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* User overridable callback: By default it calls `preStart()`.
|
* User overridable callback: By default it calls `preStart()`.
|
||||||
* @param reason the Throwable that caused the restart to happen
|
* @param reason the Throwable that caused the restart to happen
|
||||||
* <p/>
|
* <p/>
|
||||||
* Is called right AFTER restart on the newly created Actor to allow reinitialization after an Actor crash.
|
* Is called right AFTER restart on the newly created Actor to allow reinitialization after an Actor crash.
|
||||||
*/
|
*/
|
||||||
@throws(classOf[Exception])
|
@throws(classOf[Exception]) // when changing this you MUST also change UntypedActorDocTestBase
|
||||||
def postRestart(reason: Throwable) { preStart() }
|
//#lifecycle-hooks
|
||||||
|
def postRestart(reason: Throwable): Unit = {
|
||||||
|
preStart()
|
||||||
|
}
|
||||||
|
//#lifecycle-hooks
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* User overridable callback.
|
* User overridable callback.
|
||||||
|
|
|
||||||
|
|
@ -31,18 +31,12 @@ import scala.util.control.NonFatal
|
||||||
* context.actorOf(props)
|
* context.actorOf(props)
|
||||||
*
|
*
|
||||||
* // Scala
|
* // Scala
|
||||||
* context.actorOf(Props[MyActor]("name")
|
* context.actorOf(Props[MyActor])
|
||||||
* context.actorOf(Props[MyActor]
|
* context.actorOf(Props(classOf[MyActor], arg1, arg2), "name")
|
||||||
* context.actorOf(Props(new MyActor(...))
|
|
||||||
*
|
*
|
||||||
* // Java
|
* // Java
|
||||||
* context.actorOf(classOf[MyActor]);
|
* getContext().actorOf(Props.create(MyActor.class));
|
||||||
* context.actorOf(Props(new Creator<MyActor>() {
|
* getContext().actorOf(Props.create(MyActor.class, arg1, arg2), "name");
|
||||||
* public MyActor create() { ... }
|
|
||||||
* });
|
|
||||||
* context.actorOf(Props(new Creator<MyActor>() {
|
|
||||||
* public MyActor create() { ... }
|
|
||||||
* }, "name");
|
|
||||||
* }}}
|
* }}}
|
||||||
*
|
*
|
||||||
* Where no name is given explicitly, one will be automatically generated.
|
* Where no name is given explicitly, one will be automatically generated.
|
||||||
|
|
@ -534,7 +528,7 @@ private[akka] class ActorCell(
|
||||||
contextStack.set(this :: contextStack.get)
|
contextStack.set(this :: contextStack.get)
|
||||||
try {
|
try {
|
||||||
behaviorStack = emptyBehaviorStack
|
behaviorStack = emptyBehaviorStack
|
||||||
val instance = props.creator.apply()
|
val instance = props.newActor()
|
||||||
|
|
||||||
if (instance eq null)
|
if (instance eq null)
|
||||||
throw ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'")
|
throw ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'")
|
||||||
|
|
|
||||||
|
|
@ -46,7 +46,7 @@ import akka.event.LoggingAdapter
|
||||||
*
|
*
|
||||||
* public class ExampleActor Extends UntypedActor {
|
* public class ExampleActor Extends UntypedActor {
|
||||||
* // this child will be destroyed and re-created upon restart by default
|
* // this child will be destroyed and re-created upon restart by default
|
||||||
* final ActorRef other = getContext().actorOf(new Props(OtherActor.class), "childName");
|
* final ActorRef other = getContext().actorOf(Props.create(OtherActor.class), "childName");
|
||||||
*
|
*
|
||||||
* @Override
|
* @Override
|
||||||
* public void onReceive(Object o) {
|
* public void onReceive(Object o) {
|
||||||
|
|
|
||||||
|
|
@ -362,6 +362,73 @@ private[akka] object SystemGuardian {
|
||||||
case object TerminationHookDone
|
case object TerminationHookDone
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private[akka] object LocalActorRefProvider {
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Root and user guardian
|
||||||
|
*/
|
||||||
|
private class Guardian(override val supervisorStrategy: SupervisorStrategy) extends Actor {
|
||||||
|
|
||||||
|
def receive = {
|
||||||
|
case Terminated(_) ⇒ context.stop(self)
|
||||||
|
case StopChild(child) ⇒ context.stop(child)
|
||||||
|
case m ⇒ context.system.deadLetters forward DeadLetter(m, sender, self)
|
||||||
|
}
|
||||||
|
|
||||||
|
// guardian MUST NOT lose its children during restart
|
||||||
|
override def preRestart(cause: Throwable, msg: Option[Any]) {}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* System guardian
|
||||||
|
*/
|
||||||
|
private class SystemGuardian(override val supervisorStrategy: SupervisorStrategy, val guardian: ActorRef) extends Actor {
|
||||||
|
import SystemGuardian._
|
||||||
|
|
||||||
|
var terminationHooks = Set.empty[ActorRef]
|
||||||
|
|
||||||
|
def receive = {
|
||||||
|
case Terminated(`guardian`) ⇒
|
||||||
|
// time for the systemGuardian to stop, but first notify all the
|
||||||
|
// termination hooks, they will reply with TerminationHookDone
|
||||||
|
// and when all are done the systemGuardian is stopped
|
||||||
|
context.become(terminating)
|
||||||
|
terminationHooks foreach { _ ! TerminationHook }
|
||||||
|
stopWhenAllTerminationHooksDone()
|
||||||
|
case Terminated(a) ⇒
|
||||||
|
// a registered, and watched termination hook terminated before
|
||||||
|
// termination process of guardian has started
|
||||||
|
terminationHooks -= a
|
||||||
|
case StopChild(child) ⇒ context.stop(child)
|
||||||
|
case RegisterTerminationHook if sender != context.system.deadLetters ⇒
|
||||||
|
terminationHooks += sender
|
||||||
|
context watch sender
|
||||||
|
case m ⇒ context.system.deadLetters forward DeadLetter(m, sender, self)
|
||||||
|
}
|
||||||
|
|
||||||
|
def terminating: Receive = {
|
||||||
|
case Terminated(a) ⇒ stopWhenAllTerminationHooksDone(a)
|
||||||
|
case TerminationHookDone ⇒ stopWhenAllTerminationHooksDone(sender)
|
||||||
|
case m ⇒ context.system.deadLetters forward DeadLetter(m, sender, self)
|
||||||
|
}
|
||||||
|
|
||||||
|
def stopWhenAllTerminationHooksDone(remove: ActorRef): Unit = {
|
||||||
|
terminationHooks -= remove
|
||||||
|
stopWhenAllTerminationHooksDone()
|
||||||
|
}
|
||||||
|
|
||||||
|
def stopWhenAllTerminationHooksDone(): Unit =
|
||||||
|
if (terminationHooks.isEmpty) {
|
||||||
|
context.system.eventStream.stopDefaultLoggers(context.system)
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
|
||||||
|
// guardian MUST NOT lose its children during restart
|
||||||
|
override def preRestart(cause: Throwable, msg: Option[Any]) {}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Local ActorRef provider.
|
* Local ActorRef provider.
|
||||||
*
|
*
|
||||||
|
|
@ -428,82 +495,22 @@ private[akka] class LocalActorRefProvider private[akka] (
|
||||||
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = stopped.ifOff(message match {
|
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = stopped.ifOff(message match {
|
||||||
case null ⇒ throw new InvalidMessageException("Message is null")
|
case null ⇒ throw new InvalidMessageException("Message is null")
|
||||||
case NullMessage ⇒ // do nothing
|
case NullMessage ⇒ // do nothing
|
||||||
case _ ⇒ log.error(this + " received unexpected message [" + message + "]")
|
case _ ⇒ log.error(s"$this received unexpected message [$message]")
|
||||||
})
|
})
|
||||||
|
|
||||||
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
|
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
|
||||||
message match {
|
message match {
|
||||||
case Failed(child, ex, _) ⇒ { causeOfTermination = Some(ex); child.asInstanceOf[InternalActorRef].stop() }
|
case Failed(child, ex, _) ⇒
|
||||||
case _: Supervise ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead
|
log.error(ex, s"guardian $child failed, shutting down!")
|
||||||
|
causeOfTermination = Some(ex)
|
||||||
|
child.asInstanceOf[InternalActorRef].stop()
|
||||||
|
case Supervise(_, _) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead
|
||||||
case _: DeathWatchNotification ⇒ stop()
|
case _: DeathWatchNotification ⇒ stop()
|
||||||
case _ ⇒ log.error(this + " received unexpected system message [" + message + "]")
|
case _ ⇒ log.error(s"$this received unexpected system message [$message]")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
|
||||||
* Root and user guardian
|
|
||||||
*/
|
|
||||||
private class Guardian(override val supervisorStrategy: SupervisorStrategy) extends Actor {
|
|
||||||
|
|
||||||
def receive = {
|
|
||||||
case Terminated(_) ⇒ context.stop(self)
|
|
||||||
case StopChild(child) ⇒ context.stop(child)
|
|
||||||
case m ⇒ deadLetters forward DeadLetter(m, sender, self)
|
|
||||||
}
|
|
||||||
|
|
||||||
// guardian MUST NOT lose its children during restart
|
|
||||||
override def preRestart(cause: Throwable, msg: Option[Any]) {}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* System guardian
|
|
||||||
*/
|
|
||||||
private class SystemGuardian(override val supervisorStrategy: SupervisorStrategy) extends Actor {
|
|
||||||
import SystemGuardian._
|
|
||||||
|
|
||||||
var terminationHooks = Set.empty[ActorRef]
|
|
||||||
|
|
||||||
def receive = {
|
|
||||||
case Terminated(`guardian`) ⇒
|
|
||||||
// time for the systemGuardian to stop, but first notify all the
|
|
||||||
// termination hooks, they will reply with TerminationHookDone
|
|
||||||
// and when all are done the systemGuardian is stopped
|
|
||||||
context.become(terminating)
|
|
||||||
terminationHooks foreach { _ ! TerminationHook }
|
|
||||||
stopWhenAllTerminationHooksDone()
|
|
||||||
case Terminated(a) ⇒
|
|
||||||
// a registered, and watched termination hook terminated before
|
|
||||||
// termination process of guardian has started
|
|
||||||
terminationHooks -= a
|
|
||||||
case StopChild(child) ⇒ context.stop(child)
|
|
||||||
case RegisterTerminationHook if sender != context.system.deadLetters ⇒
|
|
||||||
terminationHooks += sender
|
|
||||||
context watch sender
|
|
||||||
case m ⇒ deadLetters forward DeadLetter(m, sender, self)
|
|
||||||
}
|
|
||||||
|
|
||||||
def terminating: Receive = {
|
|
||||||
case Terminated(a) ⇒ stopWhenAllTerminationHooksDone(a)
|
|
||||||
case TerminationHookDone ⇒ stopWhenAllTerminationHooksDone(sender)
|
|
||||||
case m ⇒ deadLetters forward DeadLetter(m, sender, self)
|
|
||||||
}
|
|
||||||
|
|
||||||
def stopWhenAllTerminationHooksDone(remove: ActorRef): Unit = {
|
|
||||||
terminationHooks -= remove
|
|
||||||
stopWhenAllTerminationHooksDone()
|
|
||||||
}
|
|
||||||
|
|
||||||
def stopWhenAllTerminationHooksDone(): Unit =
|
|
||||||
if (terminationHooks.isEmpty) {
|
|
||||||
eventStream.stopDefaultLoggers(system)
|
|
||||||
context.stop(self)
|
|
||||||
}
|
|
||||||
|
|
||||||
// guardian MUST NOT lose its children during restart
|
|
||||||
override def preRestart(cause: Throwable, msg: Option[Any]) {}
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* The problem is that ActorRefs need a reference to the ActorSystem to
|
* The problem is that ActorRefs need a reference to the ActorSystem to
|
||||||
* provide their service. Hence they cannot be created while the
|
* provide their service. Hence they cannot be created while the
|
||||||
|
|
@ -552,7 +559,11 @@ private[akka] class LocalActorRefProvider private[akka] (
|
||||||
protected def systemGuardianStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy
|
protected def systemGuardianStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy
|
||||||
|
|
||||||
override lazy val rootGuardian: LocalActorRef =
|
override lazy val rootGuardian: LocalActorRef =
|
||||||
new LocalActorRef(system, Props(new Guardian(rootGuardianStrategy)), theOneWhoWalksTheBubblesOfSpaceTime, rootPath) {
|
new LocalActorRef(
|
||||||
|
system,
|
||||||
|
Props(classOf[LocalActorRefProvider.Guardian], rootGuardianStrategy),
|
||||||
|
theOneWhoWalksTheBubblesOfSpaceTime,
|
||||||
|
rootPath) {
|
||||||
override def getParent: InternalActorRef = this
|
override def getParent: InternalActorRef = this
|
||||||
override def getSingleChild(name: String): InternalActorRef = name match {
|
override def getSingleChild(name: String): InternalActorRef = name match {
|
||||||
case "temp" ⇒ tempContainer
|
case "temp" ⇒ tempContainer
|
||||||
|
|
@ -568,7 +579,8 @@ private[akka] class LocalActorRefProvider private[akka] (
|
||||||
override lazy val guardian: LocalActorRef = {
|
override lazy val guardian: LocalActorRef = {
|
||||||
val cell = rootGuardian.underlying
|
val cell = rootGuardian.underlying
|
||||||
cell.reserveChild("user")
|
cell.reserveChild("user")
|
||||||
val ref = new LocalActorRef(system, Props(new Guardian(guardianStrategy)), rootGuardian, rootPath / "user")
|
val ref = new LocalActorRef(system, Props(classOf[LocalActorRefProvider.Guardian], guardianStrategy),
|
||||||
|
rootGuardian, rootPath / "user")
|
||||||
cell.initChild(ref)
|
cell.initChild(ref)
|
||||||
ref.start()
|
ref.start()
|
||||||
ref
|
ref
|
||||||
|
|
@ -577,7 +589,9 @@ private[akka] class LocalActorRefProvider private[akka] (
|
||||||
override lazy val systemGuardian: LocalActorRef = {
|
override lazy val systemGuardian: LocalActorRef = {
|
||||||
val cell = rootGuardian.underlying
|
val cell = rootGuardian.underlying
|
||||||
cell.reserveChild("system")
|
cell.reserveChild("system")
|
||||||
val ref = new LocalActorRef(system, Props(new SystemGuardian(systemGuardianStrategy)), rootGuardian, rootPath / "system")
|
val ref = new LocalActorRef(
|
||||||
|
system, Props(classOf[LocalActorRefProvider.SystemGuardian], systemGuardianStrategy, guardian),
|
||||||
|
rootGuardian, rootPath / "system")
|
||||||
cell.initChild(ref)
|
cell.initChild(ref)
|
||||||
ref.start()
|
ref.start()
|
||||||
ref
|
ref
|
||||||
|
|
@ -669,7 +683,7 @@ private[akka] class LocalActorRefProvider private[akka] (
|
||||||
|
|
||||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
||||||
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
|
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
|
||||||
props.routerConfig match {
|
props.deploy.routerConfig match {
|
||||||
case NoRouter ⇒
|
case NoRouter ⇒
|
||||||
if (settings.DebugRouterMisconfiguration) {
|
if (settings.DebugRouterMisconfiguration) {
|
||||||
deployer.lookup(path) foreach { d ⇒
|
deployer.lookup(path) foreach { d ⇒
|
||||||
|
|
|
||||||
|
|
@ -217,17 +217,11 @@ object ActorSystem {
|
||||||
*
|
*
|
||||||
* // Scala
|
* // Scala
|
||||||
* system.actorOf(Props[MyActor], "name")
|
* system.actorOf(Props[MyActor], "name")
|
||||||
* system.actorOf(Props[MyActor])
|
* system.actorOf(Props(classOf[MyActor], arg1, arg2), "name")
|
||||||
* system.actorOf(Props(new MyActor(...)))
|
|
||||||
*
|
*
|
||||||
* // Java
|
* // Java
|
||||||
* system.actorOf(MyActor.class);
|
* system.actorOf(Props.create(MyActor.class), "name");
|
||||||
* system.actorOf(Props(new Creator<MyActor>() {
|
* system.actorOf(Props.create(MyActor.class, arg1, arg2), "name");
|
||||||
* public MyActor create() { ... }
|
|
||||||
* });
|
|
||||||
* system.actorOf(Props(new Creator<MyActor>() {
|
|
||||||
* public MyActor create() { ... }
|
|
||||||
* }, "name");
|
|
||||||
* }}}
|
* }}}
|
||||||
*
|
*
|
||||||
* Where no name is given explicitly, one will be automatically generated.
|
* Where no name is given explicitly, one will be automatically generated.
|
||||||
|
|
|
||||||
|
|
@ -29,7 +29,7 @@ object Deploy {
|
||||||
* not needed when just doing deploy-as-you-go:
|
* not needed when just doing deploy-as-you-go:
|
||||||
*
|
*
|
||||||
* {{{
|
* {{{
|
||||||
* context.actorOf(someProps, "someName", Deploy(scope = RemoteScope("someOtherNodeName")))
|
* val remoteProps = someProps.withDeploy(Deploy(scope = RemoteScope("someOtherNodeName")))
|
||||||
* }}}
|
* }}}
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
|
|
@ -62,7 +62,12 @@ final case class Deploy(
|
||||||
*/
|
*/
|
||||||
def withFallback(other: Deploy): Deploy = {
|
def withFallback(other: Deploy): Deploy = {
|
||||||
val disp = if (dispatcher == Deploy.NoDispatcherGiven) other.dispatcher else dispatcher
|
val disp = if (dispatcher == Deploy.NoDispatcherGiven) other.dispatcher else dispatcher
|
||||||
Deploy(path, config.withFallback(other.config), routerConfig.withFallback(other.routerConfig), scope.withFallback(other.scope), disp)
|
Deploy(
|
||||||
|
path,
|
||||||
|
config.withFallback(other.config),
|
||||||
|
routerConfig.withFallback(other.routerConfig),
|
||||||
|
scope.withFallback(other.scope),
|
||||||
|
disp)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -115,7 +115,8 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Escalates the failure to the supervisor of the supervisor,
|
* Escalates the failure to the supervisor of the supervisor,
|
||||||
* by rethrowing the cause of the failure.
|
* by rethrowing the cause of the failure, i.e. the supervisor fails with
|
||||||
|
* the same exception as the child.
|
||||||
*/
|
*/
|
||||||
case object Escalate extends Directive
|
case object Escalate extends Directive
|
||||||
|
|
||||||
|
|
@ -132,14 +133,13 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Returning this directive stops the Actor
|
* Java API: Returning this directive stops the Actor
|
||||||
* Java API
|
|
||||||
*/
|
*/
|
||||||
def stop = Stop
|
def stop = Stop
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Returning this directive escalates the failure to the supervisor of the supervisor,
|
* Java API: Returning this directive escalates the failure to the supervisor of the supervisor,
|
||||||
* by rethrowing the cause of the failure.
|
* by rethrowing the cause of the failure, i.e. the supervisor fails with
|
||||||
* Java API
|
* the same exception as the child.
|
||||||
*/
|
*/
|
||||||
def escalate = Escalate
|
def escalate = Escalate
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -9,13 +9,16 @@ import akka.japi.Creator
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import akka.routing._
|
import akka.routing._
|
||||||
import akka.util.Reflect
|
import akka.util.Reflect
|
||||||
|
import scala.annotation.varargs
|
||||||
|
import Deploy.NoDispatcherGiven
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory for Props instances.
|
* Factory for Props instances.
|
||||||
*
|
*
|
||||||
* Props is a ActorRef configuration object, that is immutable, so it is thread safe and fully sharable.
|
* Props is a ActorRef configuration object, that is immutable, so it is thread safe and fully sharable.
|
||||||
*
|
*
|
||||||
* Used when creating new actors through; <code>ActorSystem.actorOf</code> and <code>ActorContext.actorOf</code>.
|
* Used when creating new actors through <code>ActorSystem.actorOf</code> and <code>ActorContext.actorOf</code>.
|
||||||
*/
|
*/
|
||||||
object Props {
|
object Props {
|
||||||
|
|
||||||
|
|
@ -34,30 +37,32 @@ object Props {
|
||||||
*/
|
*/
|
||||||
final val defaultDeploy = Deploy()
|
final val defaultDeploy = Deploy()
|
||||||
|
|
||||||
/**
|
|
||||||
* A Props instance whose creator will create an actor that doesn't respond to any message
|
|
||||||
*/
|
|
||||||
final val empty = new Props(() ⇒ new Actor { def receive = Actor.emptyBehavior })
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The default Props instance, uses the settings from the Props object starting with default*.
|
* The default Props instance, uses the settings from the Props object starting with default*.
|
||||||
*/
|
*/
|
||||||
final val default = new Props()
|
final val default = new Props()
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A Props instance whose creator will create an actor that doesn't respond to any message
|
||||||
|
*/
|
||||||
|
final val empty = Props[EmptyActor]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*
|
||||||
|
* (Not because it is so immensely complicated, only because we might remove it if no longer needed internally)
|
||||||
|
*/
|
||||||
|
private[akka] class EmptyActor extends Actor {
|
||||||
|
def receive = Actor.emptyBehavior
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
||||||
* of the supplied type using the default constructor.
|
* of the supplied type using the default constructor.
|
||||||
*
|
*
|
||||||
* Scala API.
|
* Scala API.
|
||||||
*/
|
*/
|
||||||
def apply[T <: Actor: ClassTag](): Props =
|
def apply[T <: Actor: ClassTag](): Props = apply(implicitly[ClassTag[T]].runtimeClass)
|
||||||
default.withCreator(implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[_ <: Actor]])
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
|
||||||
* of the supplied class using the default constructor.
|
|
||||||
*/
|
|
||||||
def apply(actorClass: Class[_ <: Actor]): Props = default.withCreator(actorClass)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
||||||
|
|
@ -65,116 +70,242 @@ object Props {
|
||||||
*
|
*
|
||||||
* Scala API.
|
* Scala API.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("give class and arguments instead", "2.2")
|
||||||
def apply(creator: ⇒ Actor): Props = default.withCreator(creator)
|
def apply(creator: ⇒ Actor): Props = default.withCreator(creator)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
||||||
* using the supplied thunk.
|
* using the supplied thunk.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("give class and arguments instead", "2.2")
|
||||||
def apply(creator: Creator[_ <: Actor]): Props = default.withCreator(creator.create)
|
def apply(creator: Creator[_ <: Actor]): Props = default.withCreator(creator.create)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The deprecated legacy constructor.
|
||||||
|
*/
|
||||||
|
@deprecated("give class and arguments instead", "2.2")
|
||||||
|
def apply(
|
||||||
|
creator: () ⇒ Actor = Props.defaultCreator,
|
||||||
|
dispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
|
routerConfig: RouterConfig = Props.defaultRoutedProps,
|
||||||
|
deploy: Deploy = Props.defaultDeploy): Props = {
|
||||||
|
|
||||||
|
val d1 = if (dispatcher != Dispatchers.DefaultDispatcherId) 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
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scala API: create a Props given a class and its constructor arguments.
|
||||||
|
*/
|
||||||
|
def apply(clazz: Class[_], args: Any*): Props = apply(defaultDeploy, clazz, args.toVector)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API: create a Props given a class and its constructor arguments.
|
||||||
|
*/
|
||||||
|
@varargs
|
||||||
|
def create(clazz: Class[_], args: AnyRef*): Props = apply(defaultDeploy, clazz, args.toVector)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Props is a ActorRef configuration object, that is immutable, so it is thread safe and fully sharable.
|
* Props is a configuration object using in creating an [[Actor]]; it is
|
||||||
* Used when creating new actors through; <code>ActorSystem.actorOf</code> and <code>ActorContext.actorOf</code>.
|
* immutable, so it is thread-safe and fully shareable.
|
||||||
*
|
|
||||||
* In case of providing code which creates the actual Actor instance, that must not return the same instance multiple times.
|
|
||||||
*
|
*
|
||||||
* Examples on Scala API:
|
* Examples on Scala API:
|
||||||
* {{{
|
* {{{
|
||||||
|
* val props = Props.empty
|
||||||
* val props = Props[MyActor]
|
* val props = Props[MyActor]
|
||||||
* val props = Props(new MyActor)
|
* val props = Props(classOf[MyActor], arg1, arg2)
|
||||||
* val props = Props(
|
*
|
||||||
* creator = ..,
|
* val otherProps = props.withDispatcher("dispatcher-id")
|
||||||
* dispatcher = ..,
|
* val otherProps = props.withDeploy(<deployment info>)
|
||||||
* routerConfig = ..
|
|
||||||
* )
|
|
||||||
* val props = Props().withCreator(new MyActor)
|
|
||||||
* val props = Props[MyActor].withRouter(RoundRobinRouter(..))
|
|
||||||
* }}}
|
* }}}
|
||||||
*
|
*
|
||||||
* Examples on Java API:
|
* Examples on Java API:
|
||||||
* {{{
|
* {{{
|
||||||
* Props props = new Props();
|
* final Props props = Props.empty();
|
||||||
* Props props = new Props(MyActor.class);
|
* final Props props = Props.create(MyActor.class, arg1, arg2);
|
||||||
* Props props = new Props(new UntypedActorFactory() {
|
*
|
||||||
* public UntypedActor create() {
|
* final Props otherProps = props.withDispatcher("dispatcher-id");
|
||||||
* return new MyActor();
|
* final Props otherProps = props.withDeploy(<deployment info>);
|
||||||
* }
|
|
||||||
* });
|
|
||||||
* Props props = new Props().withCreator(new UntypedActorFactory() { ... });
|
|
||||||
* Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..));
|
|
||||||
* }}}
|
* }}}
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(2L)
|
||||||
case class Props(
|
case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]) {
|
||||||
creator: () ⇒ Actor = Props.defaultCreator,
|
|
||||||
dispatcher: String = Dispatchers.DefaultDispatcherId,
|
// validate constructor signature; throws IllegalArgumentException if invalid
|
||||||
routerConfig: RouterConfig = Props.defaultRoutedProps,
|
Reflect.findConstructor(clazz, args)
|
||||||
deploy: Deploy = Props.defaultDeploy) {
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* No-args constructor that sets all the default values.
|
* No-args constructor that sets all the default values.
|
||||||
|
*
|
||||||
|
* @deprecated use `Props.create(clazz, args ...)` instead
|
||||||
*/
|
*/
|
||||||
def this() = this(
|
@deprecated("use Props.create()", "2.2")
|
||||||
creator = Props.defaultCreator,
|
def this() = this(Props.defaultDeploy, classOf[CreatorFunctionConsumer], Vector(Props.defaultCreator))
|
||||||
dispatcher = Dispatchers.DefaultDispatcherId)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: create Props from an [[UntypedActorFactory]]
|
* 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
|
||||||
*/
|
*/
|
||||||
def this(factory: UntypedActorFactory) = this(
|
@deprecated("use constructor which takes the actor class directly", "2.2")
|
||||||
creator = () ⇒ factory.create(),
|
def this(factory: UntypedActorFactory) = this(Props.defaultDeploy, classOf[UntypedActorFactoryConsumer], Vector(factory))
|
||||||
dispatcher = Dispatchers.DefaultDispatcherId)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: create Props from a given [[Class]]
|
* Java API: create Props from a given [[Class]]
|
||||||
|
*
|
||||||
|
* @deprecated use Props.create(clazz) instead; deprecated since it duplicates
|
||||||
|
* another API
|
||||||
*/
|
*/
|
||||||
def this(actorClass: Class[_ <: Actor]) = this(
|
@deprecated("use Props.create()", "2.2")
|
||||||
creator = FromClassCreator(actorClass),
|
def this(actorClass: Class[_ <: Actor]) = this(Props.defaultDeploy, actorClass, Vector.empty)
|
||||||
dispatcher = Dispatchers.DefaultDispatcherId,
|
|
||||||
routerConfig = Props.defaultRoutedProps)
|
@deprecated("use newActor()", "2.2")
|
||||||
|
def creator: () ⇒ Actor = newActor
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convenience method for extracting the dispatcher information from the
|
||||||
|
* contained [[Deploy]] instance.
|
||||||
|
*/
|
||||||
|
def dispatcher: String = deploy.dispatcher match {
|
||||||
|
case NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId
|
||||||
|
case x ⇒ x
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convenience method for extracting the router configuration from the
|
||||||
|
* contained [[Deploy]] instance.
|
||||||
|
*/
|
||||||
|
def routerConfig: RouterConfig = deploy.routerConfig
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Returns a new Props with the specified creator set.
|
* Scala API: Returns a new Props with the specified creator set.
|
||||||
*
|
*
|
||||||
* The creator must not return the same instance multiple times.
|
* The creator must not return the same instance multiple times.
|
||||||
*/
|
*/
|
||||||
def withCreator(c: ⇒ Actor): Props = copy(creator = () ⇒ c)
|
@deprecated("move actor into named class and use withCreator(clazz)", "2.2")
|
||||||
|
def withCreator(c: ⇒ Actor): Props = copy(clazz = classOf[CreatorFunctionConsumer], args = Vector(() ⇒ c))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Returns a new Props with the specified creator set.
|
* Java API: Returns a new Props with the specified creator set.
|
||||||
*
|
*
|
||||||
* The creator must not return the same instance multiple times.
|
* 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
|
||||||
*/
|
*/
|
||||||
def withCreator(c: Creator[Actor]): Props = copy(creator = () ⇒ c.create)
|
@deprecated("use Props.create(clazz, args ...) instead", "2.2")
|
||||||
|
def withCreator(c: Creator[Actor]): Props = copy(clazz = classOf[CreatorConsumer], args = Vector(c))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Returns a new Props with the specified creator set.
|
* Returns a new Props with the specified creator set.
|
||||||
|
*
|
||||||
|
* @deprecated use Props.create(clazz) instead; deprecated since it duplicates
|
||||||
|
* another API
|
||||||
*/
|
*/
|
||||||
def withCreator(c: Class[_ <: Actor]): Props = copy(creator = FromClassCreator(c))
|
@deprecated("use Props(clazz, args).withDeploy(other.deploy)", "2.2")
|
||||||
|
def withCreator(c: Class[_ <: Actor]): Props = copy(clazz = c, args = Vector.empty)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a new Props with the specified dispatcher set.
|
* Returns a new Props with the specified dispatcher set.
|
||||||
*/
|
*/
|
||||||
def withDispatcher(d: String): Props = copy(dispatcher = d)
|
def withDispatcher(d: String): Props = copy(deploy = deploy.copy(dispatcher = d))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a new Props with the specified router config set.
|
* Returns a new Props with the specified router config set.
|
||||||
*/
|
*/
|
||||||
def withRouter(r: RouterConfig): Props = copy(routerConfig = r)
|
def withRouter(r: RouterConfig): Props = copy(deploy = deploy.copy(routerConfig = r))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a new Props with the specified deployment configuration.
|
* Returns a new Props with the specified deployment configuration.
|
||||||
*/
|
*/
|
||||||
def withDeploy(d: Deploy): Props = copy(deploy = d)
|
def withDeploy(d: Deploy): Props = copy(deploy = d withFallback deploy)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new actor instance. This method is only useful when called during
|
||||||
|
* actor creation by the ActorSystem, i.e. for user-level code it can only be
|
||||||
|
* used within the implementation of [[IndirectActorProducer#produce]].
|
||||||
|
*/
|
||||||
|
def newActor(): Actor = {
|
||||||
|
Reflect.instantiate(clazz, args) match {
|
||||||
|
case a: Actor ⇒ a
|
||||||
|
case i: IndirectActorProducer ⇒ i.produce()
|
||||||
|
case _ ⇒ throw new IllegalArgumentException(s"unknown actor creator [$clazz]")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Obtain an upper-bound approximation of the actor class which is going to
|
||||||
|
* be created by these Props. In other words, the [[#newActor]] method will
|
||||||
|
* produce an instance of this class or a subclass thereof. This is used by
|
||||||
|
* the actor system to select special dispatchers or mailboxes in case
|
||||||
|
* dependencies are encoded in the actor type.
|
||||||
|
*/
|
||||||
|
def actorClass(): Class[_ <: Actor] = {
|
||||||
|
if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
|
||||||
|
Reflect.instantiate(clazz, args).asInstanceOf[IndirectActorProducer].actorClass
|
||||||
|
} else if (classOf[Actor].isAssignableFrom(clazz)) {
|
||||||
|
clazz.asInstanceOf[Class[_ <: Actor]]
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("unknown actor creator [$clazz]")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Used when creating an Actor from a class. Special Function0 to be
|
* This interface defines a class of actor creation strategies deviating from
|
||||||
* able to optimize serialization.
|
* the usual default of just reflectively instantiating the [[Actor]]
|
||||||
|
* subclass. It can be used to allow a dependency injection framework to
|
||||||
|
* determine the actual actor class and how it shall be instantiated.
|
||||||
*/
|
*/
|
||||||
private[akka] case class FromClassCreator(clazz: Class[_ <: Actor]) extends Function0[Actor] {
|
trait IndirectActorProducer {
|
||||||
def apply(): Actor = Reflect.instantiate(clazz)
|
|
||||||
|
/**
|
||||||
|
* This factory method must produce a fresh actor instance upon each
|
||||||
|
* invocation. <b>It is not permitted to return the same instance more than
|
||||||
|
* once.</b>
|
||||||
|
*/
|
||||||
|
def produce(): Actor
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method is used by [[Props]] to determine the type of actor which will
|
||||||
|
* be created. This means that an instance of this `IndirectActorProducer`
|
||||||
|
* will be created in order to call this method during any call to
|
||||||
|
* [[Props#actorClass]]; it should be noted that such calls may
|
||||||
|
* performed during actor set-up before the actual actor’s instantiation, and
|
||||||
|
* that the instance created for calling `actorClass` is not necessarily reused
|
||||||
|
* later to produce the actor.
|
||||||
|
*/
|
||||||
|
def actorClass: Class[_ <: Actor]
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
private[akka] class UntypedActorFactoryConsumer(factory: UntypedActorFactory) extends IndirectActorProducer {
|
||||||
|
override def actorClass = classOf[Actor]
|
||||||
|
override def produce() = factory.create()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
private[akka] class CreatorFunctionConsumer(creator: () ⇒ Actor) extends IndirectActorProducer {
|
||||||
|
override def actorClass = classOf[Actor]
|
||||||
|
override def produce() = creator()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
private[akka] class CreatorConsumer(creator: Creator[Actor]) extends IndirectActorProducer {
|
||||||
|
override def actorClass = classOf[Actor]
|
||||||
|
override def produce() = creator.create()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -71,11 +71,11 @@ import akka.japi.{ Creator }
|
||||||
*
|
*
|
||||||
* } else if (msg.equals("ErrorKernelWithDirectReply")) {
|
* } else if (msg.equals("ErrorKernelWithDirectReply")) {
|
||||||
* // Send work to one-off child which will reply directly to original sender
|
* // Send work to one-off child which will reply directly to original sender
|
||||||
* getContext().actorOf(new Props(Worker.class)).tell("DoSomeDangerousWork", getSender());
|
* getContext().actorOf(Props.create(Worker.class)).tell("DoSomeDangerousWork", getSender());
|
||||||
*
|
*
|
||||||
* } else if (msg.equals("ErrorKernelWithReplyHere")) {
|
* } else if (msg.equals("ErrorKernelWithReplyHere")) {
|
||||||
* // Send work to one-off child and collect the answer, reply handled further down
|
* // Send work to one-off child and collect the answer, reply handled further down
|
||||||
* getContext().actorOf(new Props(Worker.class)).tell("DoWorkAndReplyToMe");
|
* getContext().actorOf(Props.create(Worker.class)).tell("DoWorkAndReplyToMe", getSelf());
|
||||||
*
|
*
|
||||||
* } else throw new IllegalArgumentException("Unknown message: " + message);
|
* } else throw new IllegalArgumentException("Unknown message: " + message);
|
||||||
*
|
*
|
||||||
|
|
@ -166,4 +166,5 @@ abstract class UntypedActor extends Actor {
|
||||||
/**
|
/**
|
||||||
* Factory closure for an UntypedActor, to be used with 'Actors.actorOf(factory)'.
|
* Factory closure for an UntypedActor, to be used with 'Actors.actorOf(factory)'.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("use Props.create(clazz, args) instead", "2.2")
|
||||||
trait UntypedActorFactory extends Creator[Actor] with Serializable
|
trait UntypedActorFactory extends Creator[Actor] with Serializable
|
||||||
|
|
|
||||||
|
|
@ -168,9 +168,11 @@ private[akka] trait Children { this: ActorCell ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
private def makeChild(cell: ActorCell, props: Props, name: String, async: Boolean, systemService: Boolean): ActorRef = {
|
private def makeChild(cell: ActorCell, props: Props, name: String, async: Boolean, systemService: Boolean): ActorRef = {
|
||||||
if (cell.system.settings.SerializeAllCreators && !props.creator.isInstanceOf[NoSerializationVerificationNeeded]) {
|
if (cell.system.settings.SerializeAllCreators) {
|
||||||
val ser = SerializationExtension(cell.system)
|
val ser = SerializationExtension(cell.system)
|
||||||
ser.deserialize(ser.serialize(props.creator).get, props.creator.getClass).get
|
props.args forall (arg ⇒
|
||||||
|
arg.isInstanceOf[NoSerializationVerificationNeeded] ||
|
||||||
|
ser.deserialize(ser.serialize(arg.asInstanceOf[AnyRef]).get, arg.getClass).get != null)
|
||||||
}
|
}
|
||||||
/*
|
/*
|
||||||
* in case we are currently terminating, fail external attachChild requests
|
* in case we are currently terminating, fail external attachChild requests
|
||||||
|
|
|
||||||
|
|
@ -39,11 +39,20 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private[akka] object RoutedActorCell {
|
||||||
|
class RouterCreator(routerConfig: RouterConfig) extends IndirectActorProducer {
|
||||||
|
override def actorClass = classOf[Router]
|
||||||
|
override def produce() = routerConfig.createActor()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef)
|
private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef)
|
||||||
extends ActorCell(
|
extends ActorCell(
|
||||||
_system,
|
_system,
|
||||||
_ref,
|
_ref,
|
||||||
_props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
|
_props.copy(
|
||||||
|
deploy = _props.deploy.copy(dispatcher = _props.routerConfig.routerDispatcher),
|
||||||
|
classOf[RoutedActorCell.RouterCreator], Vector(_props.routerConfig)),
|
||||||
_supervisor) {
|
_supervisor) {
|
||||||
|
|
||||||
private[akka] val routerConfig = _props.routerConfig
|
private[akka] val routerConfig = _props.routerConfig
|
||||||
|
|
@ -59,7 +68,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
||||||
def routeeProvider = _routeeProvider
|
def routeeProvider = _routeeProvider
|
||||||
|
|
||||||
val route = {
|
val route = {
|
||||||
val routeeProps = _props.copy(routerConfig = NoRouter)
|
val routeeProps = _props.withRouter(NoRouter)
|
||||||
_routeeProvider = routerConfig.createRouteeProvider(this, routeeProps)
|
_routeeProvider = routerConfig.createRouteeProvider(this, routeeProps)
|
||||||
val r = routerConfig.createRoute(routeeProvider)
|
val r = routerConfig.createRoute(routeeProvider)
|
||||||
// initial resize, before message send
|
// initial resize, before message send
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,8 @@
|
||||||
*/
|
*/
|
||||||
package akka.util
|
package akka.util
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
|
import java.lang.reflect.Constructor
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Collection of internal reflection utilities which may or may not be
|
* Collection of internal reflection utilities which may or may not be
|
||||||
|
|
@ -43,6 +45,42 @@ private[akka] object Reflect {
|
||||||
ctor.newInstance()
|
ctor.newInstance()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
* Calls findConstructor and invokes it with the given arguments.
|
||||||
|
*/
|
||||||
|
private[akka] def instantiate[T](clazz: Class[T], args: immutable.Seq[Any]): T = {
|
||||||
|
val constructor = findConstructor(clazz, args)
|
||||||
|
constructor.setAccessible(true)
|
||||||
|
try constructor.newInstance(args.asInstanceOf[Seq[AnyRef]]: _*)
|
||||||
|
catch {
|
||||||
|
case e: IllegalArgumentException ⇒
|
||||||
|
val argString = args map (_.getClass) mkString ("[", ", ", "]")
|
||||||
|
throw new IllegalArgumentException(s"constructor $constructor is incompatible with arguments $argString", e)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
* Implements a primitive form of overload resolution a.k.a. finding the
|
||||||
|
* right constructor.
|
||||||
|
*/
|
||||||
|
private[akka] def findConstructor[T](clazz: Class[T], args: immutable.Seq[Any]): Constructor[T] = {
|
||||||
|
def error(msg: String): Nothing = {
|
||||||
|
val argClasses = args map (_.getClass) mkString ", "
|
||||||
|
throw new IllegalArgumentException(s"$msg found on $clazz for arguments [$argClasses]")
|
||||||
|
}
|
||||||
|
val candidates =
|
||||||
|
clazz.getDeclaredConstructors filter (c ⇒
|
||||||
|
c.getParameterTypes.length == args.length &&
|
||||||
|
(c.getParameterTypes zip args forall {
|
||||||
|
case (found, required) ⇒ found.isInstance(required) || BoxedType(found).isInstance(required)
|
||||||
|
}))
|
||||||
|
if (candidates.size == 1) candidates.head.asInstanceOf[Constructor[T]]
|
||||||
|
else if (candidates.size > 1) error("multiple matching constructors")
|
||||||
|
else error("no matching constructor")
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
* @param clazz the class which to instantiate an instance of
|
* @param clazz the class which to instantiate an instance of
|
||||||
|
|
|
||||||
|
|
@ -99,7 +99,7 @@ class ConsumerBroadcast(promise: Promise[(Future[List[List[ActorRef]]], Future[L
|
||||||
}
|
}
|
||||||
promise.success(Future.sequence(allActivationFutures) -> Future.sequence(allDeactivationFutures))
|
promise.success(Future.sequence(allActivationFutures) -> Future.sequence(allDeactivationFutures))
|
||||||
|
|
||||||
broadcaster = Some(context.actorOf(Props[Registrar] withRouter (BroadcastRouter(routees)), "registrarRouter"))
|
broadcaster = Some(context.actorOf(Props.empty withRouter (BroadcastRouter(routees)), "registrarRouter"))
|
||||||
case reg: Any ⇒
|
case reg: Any ⇒
|
||||||
broadcaster.foreach(_.forward(reg))
|
broadcaster.foreach(_.forward(reg))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -74,16 +74,17 @@ import ReliableProxy._
|
||||||
* situations or other VM errors).
|
* situations or other VM errors).
|
||||||
*
|
*
|
||||||
* You can create a reliable connection like this:
|
* You can create a reliable connection like this:
|
||||||
|
*
|
||||||
|
* In Scala:
|
||||||
|
*
|
||||||
* {{{
|
* {{{
|
||||||
* val proxy = context.actorOf(Props(new ReliableProxy(target)))
|
* val proxy = context.actorOf(Props(classOf[ReliableProxy], target))
|
||||||
* }}}
|
* }}}
|
||||||
* or in Java:
|
*
|
||||||
|
* In Java:
|
||||||
|
*
|
||||||
* {{{
|
* {{{
|
||||||
* final ActorRef proxy = getContext().actorOf(new Props(new UntypedActorFactory() {
|
* final ActorRef proxy = getContext().actorOf(Props.create(ReliableProxy.class target));
|
||||||
* public Actor create() {
|
|
||||||
* return new ReliableProxy(target);
|
|
||||||
* }
|
|
||||||
* }));
|
|
||||||
* }}}
|
* }}}
|
||||||
*
|
*
|
||||||
* '''''Please note:''''' the tunnel is uni-directional, and original sender
|
* '''''Please note:''''' the tunnel is uni-directional, and original sender
|
||||||
|
|
|
||||||
|
|
@ -160,13 +160,17 @@ object TimerBasedThrottler {
|
||||||
* sending out further messages:
|
* sending out further messages:
|
||||||
* {{{
|
* {{{
|
||||||
* // A simple actor that prints whatever it receives
|
* // A simple actor that prints whatever it receives
|
||||||
* val printer = system.actorOf(Props(new Actor {
|
* class Printer extends Actor {
|
||||||
* def receive = {
|
* def receive = {
|
||||||
* case x => println(x)
|
* case x => println(x)
|
||||||
* }
|
* }
|
||||||
* }))
|
* }
|
||||||
|
*
|
||||||
|
* val printer = system.actorOf(Props[Printer], "printer")
|
||||||
|
*
|
||||||
* // The throttler for this example, setting the rate
|
* // The throttler for this example, setting the rate
|
||||||
* val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second))))
|
* val throttler = system.actorOf(Props(classOf[TimerBasedThrottler], 3 msgsPer 1.second))
|
||||||
|
*
|
||||||
* // Set the target
|
* // Set the target
|
||||||
* throttler ! SetTarget(Some(printer))
|
* throttler ! SetTarget(Some(printer))
|
||||||
* // These three messages will be sent to the printer immediately
|
* // These three messages will be sent to the printer immediately
|
||||||
|
|
|
||||||
|
|
@ -23,7 +23,7 @@ a choice of the following four options:
|
||||||
#. Resume the subordinate, keeping its accumulated internal state
|
#. Resume the subordinate, keeping its accumulated internal state
|
||||||
#. Restart the subordinate, clearing out its accumulated internal state
|
#. Restart the subordinate, clearing out its accumulated internal state
|
||||||
#. Terminate the subordinate permanently
|
#. Terminate the subordinate permanently
|
||||||
#. Escalate the failure
|
#. Escalate the failure, thereby failing itself
|
||||||
|
|
||||||
It is important to always view an actor as part of a supervision hierarchy,
|
It is important to always view an actor as part of a supervision hierarchy,
|
||||||
which explains the existence of the fourth choice (as a supervisor also is
|
which explains the existence of the fourth choice (as a supervisor also is
|
||||||
|
|
|
||||||
|
|
@ -186,7 +186,7 @@ public class FSMDocTestBase {
|
||||||
|
|
||||||
@org.junit.Test
|
@org.junit.Test
|
||||||
public void mustBunch() {
|
public void mustBunch() {
|
||||||
final ActorRef buncher = system.actorOf(new Props(MyFSM.class));
|
final ActorRef buncher = system.actorOf(Props.create(MyFSM.class));
|
||||||
final TestProbe probe = new TestProbe(system);
|
final TestProbe probe = new TestProbe(system);
|
||||||
buncher.tell(new SetTarget(probe.ref()), null);
|
buncher.tell(new SetTarget(probe.ref()), null);
|
||||||
buncher.tell(new Queue(1), null);
|
buncher.tell(new Queue(1), null);
|
||||||
|
|
|
||||||
|
|
@ -30,7 +30,6 @@ import static java.util.concurrent.TimeUnit.SECONDS;
|
||||||
import static akka.japi.Util.immutableSeq;
|
import static akka.japi.Util.immutableSeq;
|
||||||
import akka.japi.Function;
|
import akka.japi.Function;
|
||||||
import scala.Option;
|
import scala.Option;
|
||||||
import scala.collection.JavaConverters;
|
|
||||||
import scala.collection.immutable.Seq;
|
import scala.collection.immutable.Seq;
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
@ -171,10 +170,10 @@ public class FaultHandlingTestBase {
|
||||||
system.eventStream().publish(new TestEvent.Mute(ignoreExceptions));
|
system.eventStream().publish(new TestEvent.Mute(ignoreExceptions));
|
||||||
|
|
||||||
//#create
|
//#create
|
||||||
Props superprops = new Props(Supervisor.class);
|
Props superprops = Props.create(Supervisor.class);
|
||||||
ActorRef supervisor = system.actorOf(superprops, "supervisor");
|
ActorRef supervisor = system.actorOf(superprops, "supervisor");
|
||||||
ActorRef child = (ActorRef) Await.result(ask(supervisor,
|
ActorRef child = (ActorRef) Await.result(ask(supervisor,
|
||||||
new Props(Child.class), 5000), timeout);
|
Props.create(Child.class), 5000), timeout);
|
||||||
//#create
|
//#create
|
||||||
|
|
||||||
//#resume
|
//#resume
|
||||||
|
|
@ -198,7 +197,7 @@ public class FaultHandlingTestBase {
|
||||||
|
|
||||||
//#escalate-kill
|
//#escalate-kill
|
||||||
child = (ActorRef) Await.result(ask(supervisor,
|
child = (ActorRef) Await.result(ask(supervisor,
|
||||||
new Props(Child.class), 5000), timeout);
|
Props.create(Child.class), 5000), timeout);
|
||||||
probe.watch(child);
|
probe.watch(child);
|
||||||
assert Await.result(ask(child, "get", 5000), timeout).equals(0);
|
assert Await.result(ask(child, "get", 5000), timeout).equals(0);
|
||||||
child.tell(new Exception(), null);
|
child.tell(new Exception(), null);
|
||||||
|
|
@ -206,10 +205,10 @@ public class FaultHandlingTestBase {
|
||||||
//#escalate-kill
|
//#escalate-kill
|
||||||
|
|
||||||
//#escalate-restart
|
//#escalate-restart
|
||||||
superprops = new Props(Supervisor2.class);
|
superprops = Props.create(Supervisor2.class);
|
||||||
supervisor = system.actorOf(superprops);
|
supervisor = system.actorOf(superprops);
|
||||||
child = (ActorRef) Await.result(ask(supervisor,
|
child = (ActorRef) Await.result(ask(supervisor,
|
||||||
new Props(Child.class), 5000), timeout);
|
Props.create(Child.class), 5000), timeout);
|
||||||
child.tell(23, null);
|
child.tell(23, null);
|
||||||
assert Await.result(ask(child, "get", 5000), timeout).equals(23);
|
assert Await.result(ask(child, "get", 5000), timeout).equals(23);
|
||||||
child.tell(new Exception(), null);
|
child.tell(new Exception(), null);
|
||||||
|
|
@ -219,6 +218,7 @@ public class FaultHandlingTestBase {
|
||||||
}
|
}
|
||||||
|
|
||||||
//#testkit
|
//#testkit
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
public <A> Seq<A> seq(A... args) {
|
public <A> Seq<A> seq(A... args) {
|
||||||
return immutableSeq(args);
|
return immutableSeq(args);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -10,7 +10,7 @@ import akka.actor.UntypedActor;
|
||||||
|
|
||||||
//#context-actorOf
|
//#context-actorOf
|
||||||
public class FirstUntypedActor extends UntypedActor {
|
public class FirstUntypedActor extends UntypedActor {
|
||||||
ActorRef myActor = getContext().actorOf(new Props(MyActor.class), "myactor");
|
ActorRef myActor = getContext().actorOf(Props.create(MyActor.class), "myactor");
|
||||||
|
|
||||||
//#context-actorOf
|
//#context-actorOf
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -78,7 +78,7 @@ public class InitializationDocSpecJava {
|
||||||
public void testIt() {
|
public void testIt() {
|
||||||
|
|
||||||
new JavaTestKit(system) {{
|
new JavaTestKit(system) {{
|
||||||
ActorRef testactor = system.actorOf(new Props(MessageInitExample.class), "testactor");
|
ActorRef testactor = system.actorOf(Props.create(MessageInitExample.class), "testactor");
|
||||||
String probe = "U OK?";
|
String probe = "U OK?";
|
||||||
|
|
||||||
testactor.tell(probe, getRef());
|
testactor.tell(probe, getRef());
|
||||||
|
|
|
||||||
|
|
@ -4,13 +4,16 @@
|
||||||
package docs.actor;
|
package docs.actor;
|
||||||
|
|
||||||
//#receive-timeout
|
//#receive-timeout
|
||||||
|
import akka.actor.ActorRef;
|
||||||
import akka.actor.ReceiveTimeout;
|
import akka.actor.ReceiveTimeout;
|
||||||
import akka.actor.UntypedActor;
|
import akka.actor.UntypedActor;
|
||||||
import scala.concurrent.duration.Duration;
|
import scala.concurrent.duration.Duration;
|
||||||
|
|
||||||
public class MyReceivedTimeoutUntypedActor extends UntypedActor {
|
public class MyReceiveTimeoutUntypedActor extends UntypedActor {
|
||||||
|
|
||||||
public MyReceivedTimeoutUntypedActor() {
|
ActorRef target = getContext().system().deadLetters();
|
||||||
|
|
||||||
|
public MyReceiveTimeoutUntypedActor() {
|
||||||
// To set an initial delay
|
// To set an initial delay
|
||||||
getContext().setReceiveTimeout(Duration.create("30 seconds"));
|
getContext().setReceiveTimeout(Duration.create("30 seconds"));
|
||||||
}
|
}
|
||||||
|
|
@ -18,12 +21,13 @@ public class MyReceivedTimeoutUntypedActor extends UntypedActor {
|
||||||
public void onReceive(Object message) {
|
public void onReceive(Object message) {
|
||||||
if (message.equals("Hello")) {
|
if (message.equals("Hello")) {
|
||||||
// To set in a response to a message
|
// To set in a response to a message
|
||||||
getContext().setReceiveTimeout(Duration.create("10 seconds"));
|
getContext().setReceiveTimeout(Duration.create("1 second"));
|
||||||
getSender().tell("Hello world", getSelf());
|
target = getSender();
|
||||||
} else if (message == ReceiveTimeout.getInstance()) {
|
target.tell("Hello world", getSelf());
|
||||||
|
} else if (message instanceof ReceiveTimeout) {
|
||||||
// To turn it off
|
// To turn it off
|
||||||
getContext().setReceiveTimeout(Duration.Undefined());
|
getContext().setReceiveTimeout(Duration.Undefined());
|
||||||
throw new RuntimeException("received timeout");
|
target.tell("timeout", getSelf());
|
||||||
} else {
|
} else {
|
||||||
unhandled(message);
|
unhandled(message);
|
||||||
}
|
}
|
||||||
|
|
@ -12,9 +12,10 @@ public class MyUntypedActor extends UntypedActor {
|
||||||
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||||
|
|
||||||
public void onReceive(Object message) throws Exception {
|
public void onReceive(Object message) throws Exception {
|
||||||
if (message instanceof String)
|
if (message instanceof String) {
|
||||||
log.info("Received String message: {}", message);
|
log.info("Received String message: {}", message);
|
||||||
else
|
getSender().tell(message, getSelf());
|
||||||
|
} else
|
||||||
unhandled(message);
|
unhandled(message);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -10,9 +10,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
//#imports1
|
//#imports1
|
||||||
|
|
||||||
//#imports2
|
//#imports2
|
||||||
import akka.actor.Actor;
|
|
||||||
import akka.actor.UntypedActor;
|
import akka.actor.UntypedActor;
|
||||||
import akka.actor.UntypedActorFactory;
|
|
||||||
import akka.actor.Cancellable;
|
import akka.actor.Cancellable;
|
||||||
//#imports2
|
//#imports2
|
||||||
|
|
||||||
|
|
@ -32,7 +30,7 @@ public class SchedulerDocTestBase {
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
|
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
|
||||||
testActor = system.actorOf(new Props(MyUntypedActor.class));
|
testActor = system.actorOf(Props.create(MyUntypedActor.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
@ -61,10 +59,8 @@ public class SchedulerDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void scheduleRecurringTask() {
|
public void scheduleRecurringTask() {
|
||||||
//#schedule-recurring
|
//#schedule-recurring
|
||||||
ActorRef tickActor = system.actorOf(new Props().withCreator(
|
class Ticker extends UntypedActor {
|
||||||
new UntypedActorFactory() {
|
@Override
|
||||||
public UntypedActor create() {
|
|
||||||
return new UntypedActor() {
|
|
||||||
public void onReceive(Object message) {
|
public void onReceive(Object message) {
|
||||||
if (message.equals("Tick")) {
|
if (message.equals("Tick")) {
|
||||||
// Do someting
|
// Do someting
|
||||||
|
|
@ -72,9 +68,9 @@ public class SchedulerDocTestBase {
|
||||||
unhandled(message);
|
unhandled(message);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
|
||||||
}
|
}
|
||||||
}));
|
|
||||||
|
ActorRef tickActor = system.actorOf(Props.create(Ticker.class, this));
|
||||||
|
|
||||||
//This will schedule to send the Tick-message
|
//This will schedule to send the Tick-message
|
||||||
//to the tickActor after 0ms repeating every 50ms
|
//to the tickActor after 0ms repeating every 50ms
|
||||||
|
|
|
||||||
|
|
@ -3,74 +3,82 @@
|
||||||
*/
|
*/
|
||||||
package docs.actor;
|
package docs.actor;
|
||||||
|
|
||||||
//#imports
|
//#import-ask
|
||||||
import akka.actor.ActorRef;
|
|
||||||
import akka.actor.ActorSystem;
|
|
||||||
import akka.actor.Props;
|
|
||||||
//#imports
|
|
||||||
|
|
||||||
//#import-future
|
|
||||||
import scala.concurrent.Future;
|
|
||||||
import akka.dispatch.Futures;
|
|
||||||
import akka.dispatch.Mapper;
|
|
||||||
import scala.concurrent.Await;
|
|
||||||
import scala.concurrent.duration.Duration;
|
|
||||||
import akka.testkit.AkkaSpec;
|
|
||||||
import akka.util.Timeout;
|
|
||||||
//#import-future
|
|
||||||
|
|
||||||
//#import-actors
|
|
||||||
import akka.actor.PoisonPill;
|
|
||||||
import akka.actor.Kill;
|
|
||||||
//#import-actors
|
|
||||||
|
|
||||||
//#import-procedure
|
|
||||||
import akka.japi.Procedure;
|
|
||||||
//#import-procedure
|
|
||||||
|
|
||||||
//#import-watch
|
|
||||||
import akka.actor.Terminated;
|
|
||||||
//#import-watch
|
|
||||||
|
|
||||||
//#import-identify
|
|
||||||
import akka.actor.ActorSelection;
|
|
||||||
import akka.actor.Identify;
|
|
||||||
import akka.actor.ActorIdentity;
|
|
||||||
//#import-identify
|
|
||||||
|
|
||||||
//#import-gracefulStop
|
|
||||||
import static akka.pattern.Patterns.gracefulStop;
|
|
||||||
import scala.concurrent.Future;
|
|
||||||
import scala.concurrent.Await;
|
|
||||||
import scala.concurrent.duration.Duration;
|
|
||||||
import akka.pattern.AskTimeoutException;
|
|
||||||
//#import-gracefulStop
|
|
||||||
|
|
||||||
//#import-askPipe
|
|
||||||
import static akka.pattern.Patterns.ask;
|
import static akka.pattern.Patterns.ask;
|
||||||
import static akka.pattern.Patterns.pipe;
|
import static akka.pattern.Patterns.pipe;
|
||||||
import scala.concurrent.Future;
|
//#import-ask
|
||||||
import akka.dispatch.Futures;
|
//#import-gracefulStop
|
||||||
import scala.concurrent.duration.Duration;
|
import static akka.pattern.Patterns.gracefulStop;
|
||||||
import akka.util.Timeout;
|
//#import-gracefulStop
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
//#import-askPipe
|
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
//#import-gracefulStop
|
||||||
|
import scala.concurrent.Await;
|
||||||
|
//#import-ask
|
||||||
|
import scala.concurrent.Future;
|
||||||
|
import scala.concurrent.duration.Duration;
|
||||||
|
//#import-ask
|
||||||
|
//#import-gracefulStop
|
||||||
|
//#import-indirect
|
||||||
|
import akka.actor.Actor;
|
||||||
|
//#import-indirect
|
||||||
|
//#import-identify
|
||||||
|
import akka.actor.ActorIdentity;
|
||||||
|
//#import-identify
|
||||||
|
import akka.actor.ActorKilledException;
|
||||||
|
//#import-identify
|
||||||
|
import akka.actor.ActorSelection;
|
||||||
|
//#import-identify
|
||||||
|
//#import-actorRef
|
||||||
|
import akka.actor.ActorRef;
|
||||||
|
import akka.actor.ActorSystem;
|
||||||
|
//#import-actorRef
|
||||||
|
//#import-identify
|
||||||
|
import akka.actor.Identify;
|
||||||
|
//#import-identify
|
||||||
|
//#import-indirect
|
||||||
|
import akka.actor.IndirectActorProducer;
|
||||||
|
//#import-indirect
|
||||||
|
import akka.actor.OneForOneStrategy;
|
||||||
|
//#import-props
|
||||||
|
import akka.actor.Props;
|
||||||
|
//#import-props
|
||||||
|
import akka.actor.SupervisorStrategy;
|
||||||
|
import akka.actor.SupervisorStrategy.Directive;
|
||||||
|
//#import-terminated
|
||||||
|
import akka.actor.Terminated;
|
||||||
|
//#import-terminated
|
||||||
|
//#import-untypedActor
|
||||||
|
import akka.actor.UntypedActor;
|
||||||
|
import akka.actor.UntypedActorFactory;
|
||||||
|
//#import-untypedActor
|
||||||
//#import-stash
|
//#import-stash
|
||||||
import akka.actor.UntypedActorWithStash;
|
import akka.actor.UntypedActorWithStash;
|
||||||
//#import-stash
|
//#import-stash
|
||||||
|
//#import-ask
|
||||||
import akka.actor.UntypedActor;
|
import akka.dispatch.Futures;
|
||||||
import akka.actor.UntypedActorFactory;
|
import akka.dispatch.Mapper;
|
||||||
|
//#import-ask
|
||||||
import org.junit.Test;
|
import akka.japi.Function;
|
||||||
import org.junit.AfterClass;
|
//#import-procedure
|
||||||
import org.junit.BeforeClass;
|
import akka.japi.Procedure;
|
||||||
import scala.Option;
|
//#import-procedure
|
||||||
import java.lang.Object;
|
//#import-gracefulStop
|
||||||
import java.util.Iterator;
|
import akka.pattern.AskTimeoutException;
|
||||||
|
//#import-gracefulStop
|
||||||
import akka.pattern.Patterns;
|
import akka.pattern.Patterns;
|
||||||
|
import akka.testkit.AkkaSpec;
|
||||||
|
import akka.testkit.JavaTestKit;
|
||||||
|
//#import-ask
|
||||||
|
import akka.util.Timeout;
|
||||||
|
//#import-ask
|
||||||
|
|
||||||
public class UntypedActorDocTestBase {
|
public class UntypedActorDocTestBase {
|
||||||
|
|
||||||
|
|
@ -87,71 +95,148 @@ public class UntypedActorDocTestBase {
|
||||||
system = null;
|
system = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void createProps() {
|
public void createProps() {
|
||||||
//#creating-props-config
|
//#creating-props-config
|
||||||
Props props1 = new Props();
|
Props props1 = Props.create(MyUntypedActor.class);
|
||||||
Props props2 = new Props(MyUntypedActor.class);
|
Props props2 = Props.create(MyActor.class, "...");
|
||||||
Props props3 = new Props(new UntypedActorFactory() {
|
|
||||||
public UntypedActor create() {
|
|
||||||
return new MyUntypedActor();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
Props props4 = props1.withCreator(new UntypedActorFactory() {
|
|
||||||
public UntypedActor create() {
|
|
||||||
return new MyUntypedActor();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
//#creating-props-config
|
//#creating-props-config
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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
|
@Test
|
||||||
public void systemActorOf() {
|
public void systemActorOf() {
|
||||||
//#system-actorOf
|
//#system-actorOf
|
||||||
ActorSystem system = ActorSystem.create("MySystem");
|
// ActorSystem is a heavy object: create only one per application
|
||||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class), "myactor");
|
final ActorSystem system = ActorSystem.create("MySystem");
|
||||||
|
final ActorRef myActor = system.actorOf(Props.create(MyUntypedActor.class), "myactor");
|
||||||
//#system-actorOf
|
//#system-actorOf
|
||||||
myActor.tell("test", null);
|
try {
|
||||||
|
new JavaTestKit(system) {
|
||||||
|
{
|
||||||
|
myActor.tell("hello", getRef());
|
||||||
|
expectMsgEquals("hello");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} finally {
|
||||||
system.shutdown();
|
system.shutdown();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void contextActorOf() {
|
public void contextActorOf() {
|
||||||
|
new JavaTestKit(system) {
|
||||||
|
{
|
||||||
//#context-actorOf
|
//#context-actorOf
|
||||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class), "myactor2");
|
class A extends UntypedActor {
|
||||||
//#context-actorOf
|
final ActorRef child =
|
||||||
myActor.tell("test", null);
|
getContext().actorOf(Props.create(MyUntypedActor.class), "myChild");
|
||||||
|
//#plus-some-behavior
|
||||||
|
@Override
|
||||||
|
public void onReceive(Object msg) {
|
||||||
|
getSender().tell(child, getSelf());
|
||||||
}
|
}
|
||||||
|
//#plus-some-behavior
|
||||||
|
}
|
||||||
|
//#context-actorOf
|
||||||
|
final ActorRef top = system.actorOf(Props.create(A.class, this));
|
||||||
|
top.tell("hello", getRef());
|
||||||
|
final ActorRef child = expectMsgClass(ActorRef.class);
|
||||||
|
child.tell("hello", getRef());
|
||||||
|
expectMsgEquals("hello");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
// this is just to make the test below a tiny fraction nicer
|
||||||
|
private ActorSystem getContext() {
|
||||||
|
return system;
|
||||||
|
}
|
||||||
|
|
||||||
|
static
|
||||||
|
//#creating-indirectly
|
||||||
|
class DependencyInjector implements IndirectActorProducer {
|
||||||
|
final Object applicationContext;
|
||||||
|
final String beanName;
|
||||||
|
|
||||||
|
public DependencyInjector(Object applicationContext, String beanName) {
|
||||||
|
this.applicationContext = applicationContext;
|
||||||
|
this.beanName = beanName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<? extends Actor> actorClass() {
|
||||||
|
return MyActor.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public MyActor produce() {
|
||||||
|
MyActor result;
|
||||||
|
//#obtain-fresh-Actor-instance-from-DI-framework
|
||||||
|
result = new MyActor((String) applicationContext);
|
||||||
|
//#obtain-fresh-Actor-instance-from-DI-framework
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
//#creating-indirectly
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void constructorActorOf() {
|
public void indirectActorOf() {
|
||||||
//#creating-constructor
|
final String applicationContext = "...";
|
||||||
// allows passing in arguments to the MyActor constructor
|
//#creating-indirectly
|
||||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
|
||||||
public UntypedActor create() {
|
final ActorRef myActor = getContext().actorOf(
|
||||||
return new MyActor("...");
|
Props.create(DependencyInjector.class, applicationContext, "MyActor"), "myactor3");
|
||||||
|
//#creating-indirectly
|
||||||
|
new JavaTestKit(system) {
|
||||||
|
{
|
||||||
|
myActor.tell("hello", getRef());
|
||||||
|
expectMsgEquals("...");
|
||||||
}
|
}
|
||||||
}), "myactor3");
|
};
|
||||||
//#creating-constructor
|
|
||||||
myActor.tell("test", null);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void propsActorOf() {
|
|
||||||
//#creating-props
|
|
||||||
ActorRef myActor = system.actorOf(
|
|
||||||
new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), "myactor4");
|
|
||||||
//#creating-props
|
|
||||||
myActor.tell("test", null);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void usingAsk() throws Exception {
|
public void usingAsk() throws Exception {
|
||||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
ActorRef myActor = system.actorOf(Props.create(MyAskActor.class, this), "myactor5");
|
||||||
public UntypedActor create() {
|
|
||||||
return new MyAskActor();
|
|
||||||
}
|
|
||||||
}), "myactor5");
|
|
||||||
|
|
||||||
//#using-ask
|
//#using-ask
|
||||||
Future<Object> future = Patterns.ask(myActor, "Hello", 1000);
|
Future<Object> future = Patterns.ask(myActor, "Hello", 1000);
|
||||||
|
|
@ -161,55 +246,181 @@ public class UntypedActorDocTestBase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void receiveTimeout() {
|
public void receiveTimeout() {
|
||||||
ActorRef myActor = system.actorOf(new Props(MyReceivedTimeoutUntypedActor.class));
|
final ActorRef myActor = system.actorOf(Props.create(MyReceiveTimeoutUntypedActor.class));
|
||||||
myActor.tell("Hello", null);
|
new JavaTestKit(system) {
|
||||||
|
{
|
||||||
|
new Within(Duration.create(1, TimeUnit.SECONDS), Duration.create(1500,
|
||||||
|
TimeUnit.MILLISECONDS)) {
|
||||||
|
@Override
|
||||||
|
protected void run() {
|
||||||
|
myActor.tell("Hello", getRef());
|
||||||
|
expectMsgEquals("Hello world");
|
||||||
|
expectMsgEquals("timeout");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void usePoisonPill() {
|
public void usePoisonPill() {
|
||||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class));
|
final ActorRef myActor = system.actorOf(Props.create(MyUntypedActor.class));
|
||||||
|
new JavaTestKit(system) {
|
||||||
|
{
|
||||||
|
final ActorRef sender = getRef();
|
||||||
//#poison-pill
|
//#poison-pill
|
||||||
myActor.tell(PoisonPill.getInstance(), null);
|
myActor.tell(akka.actor.PoisonPill.getInstance(), sender);
|
||||||
//#poison-pill
|
//#poison-pill
|
||||||
|
watch(myActor);
|
||||||
|
expectTerminated(myActor);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void useKill() {
|
public void useKill() {
|
||||||
ActorRef victim = system.actorOf(new Props(MyUntypedActor.class));
|
new JavaTestKit(system) {
|
||||||
|
{
|
||||||
|
class Master extends UntypedActor {
|
||||||
|
private SupervisorStrategy strategy = new OneForOneStrategy(-1,
|
||||||
|
Duration.Undefined(), new Function<Throwable, Directive>() {
|
||||||
|
@Override
|
||||||
|
public Directive apply(Throwable thr) {
|
||||||
|
if (thr instanceof ActorKilledException) {
|
||||||
|
target.tell("killed", getSelf());
|
||||||
|
getContext().stop(getSelf());
|
||||||
|
return SupervisorStrategy.stop();
|
||||||
|
}
|
||||||
|
return SupervisorStrategy.escalate();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
final ActorRef target;
|
||||||
|
ActorRef child;
|
||||||
|
|
||||||
|
//#preStart
|
||||||
|
@Override
|
||||||
|
public void preStart() {
|
||||||
|
child = getContext().actorOf(Props.empty());
|
||||||
|
}
|
||||||
|
//#preStart
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
public Master(ActorRef target) {
|
||||||
|
this.target = target;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Only compilation of `forward` is verified here.
|
||||||
|
*/
|
||||||
|
final Object result = "";
|
||||||
|
//#forward
|
||||||
|
target.forward(result, getContext());
|
||||||
|
//#forward
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SupervisorStrategy supervisorStrategy() {
|
||||||
|
return strategy;
|
||||||
|
}
|
||||||
|
|
||||||
|
//#reply
|
||||||
|
@Override
|
||||||
|
public void onReceive(Object msg) {
|
||||||
|
Object result =
|
||||||
|
//#calculate-result
|
||||||
|
child;
|
||||||
|
//#calculate-result
|
||||||
|
|
||||||
|
// do not forget the second argument!
|
||||||
|
getSender().tell(result, getSelf());
|
||||||
|
}
|
||||||
|
//#reply
|
||||||
|
|
||||||
|
//#postStop
|
||||||
|
@Override
|
||||||
|
public void postStop() {
|
||||||
|
//#clean-up-resources-here
|
||||||
|
final String message = "stopped";
|
||||||
|
//#tell
|
||||||
|
// don’t forget to think about who is the sender (2nd argument)
|
||||||
|
target.tell(message, getSelf());
|
||||||
|
//#tell
|
||||||
|
//#clean-up-resources-here
|
||||||
|
}
|
||||||
|
//#postStop
|
||||||
|
}
|
||||||
|
final ActorRef master = system.actorOf(Props.create(Master.class, this, getRef()));
|
||||||
|
expectMsgEquals("");
|
||||||
|
master.tell("", getRef());
|
||||||
|
final ActorRef victim = expectMsgClass(ActorRef.class);
|
||||||
//#kill
|
//#kill
|
||||||
victim.tell(Kill.getInstance(), null);
|
victim.tell(akka.actor.Kill.getInstance(), null);
|
||||||
//#kill
|
//#kill
|
||||||
|
expectMsgEquals("killed");
|
||||||
|
expectMsgEquals("stopped");
|
||||||
|
assert getLastSender().equals(master);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void useBecome() {
|
public void useBecome() {
|
||||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
new JavaTestKit(system) {
|
||||||
public UntypedActor create() {
|
{
|
||||||
return new HotSwapActor();
|
ActorRef myActor = system.actorOf(Props.create(HotSwapActor.class));
|
||||||
|
myActor.tell("foo", getRef());
|
||||||
|
myActor.tell("bar", getRef());
|
||||||
|
expectMsgEquals("I am already happy :-)");
|
||||||
|
myActor.tell("bar", getRef());
|
||||||
|
expectMsgEquals("I am already happy :-)");
|
||||||
}
|
}
|
||||||
}));
|
};
|
||||||
myActor.tell("foo", null);
|
|
||||||
myActor.tell("bar", null);
|
|
||||||
myActor.tell("bar", null);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void useWatch() throws Exception {
|
public void useWatch() throws Exception {
|
||||||
ActorRef myActor = system.actorOf(new Props(WatchActor.class));
|
ActorRef myActor = system.actorOf(Props.create(WatchActor.class));
|
||||||
Future<Object> future = Patterns.ask(myActor, "kill", 1000);
|
Future<Object> future = Patterns.ask(myActor, "kill", 1000);
|
||||||
assert Await.result(future, Duration.create("1 second")).equals("finished");
|
assert Await.result(future, Duration.create("1 second")).equals("finished");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// compilation test only
|
||||||
|
public void compileSelections() {
|
||||||
|
//#selection-local
|
||||||
|
// will look up this absolute path
|
||||||
|
getContext().actorSelection("/user/serviceA/actor");
|
||||||
|
// will look up sibling beneath same supervisor
|
||||||
|
getContext().actorSelection("../joe");
|
||||||
|
//#selection-local
|
||||||
|
|
||||||
|
//#selection-wildcard
|
||||||
|
// will look all children to serviceB with names starting with worker
|
||||||
|
getContext().actorSelection("/user/serviceB/worker*");
|
||||||
|
// will look up all siblings beneath same supervisor
|
||||||
|
getContext().actorSelection("../*");
|
||||||
|
//#selection-wildcard
|
||||||
|
|
||||||
|
//#selection-remote
|
||||||
|
getContext().actorSelection("akka.tcp://app@otherhost:1234/user/serviceB");
|
||||||
|
//#selection-remote
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void useIdentify() throws Exception {
|
public void useIdentify() throws Exception {
|
||||||
ActorRef a = system.actorOf(new Props(MyUntypedActor.class), "another");
|
new JavaTestKit(system) {
|
||||||
ActorRef b = system.actorOf(new Props(Follower.class));
|
{
|
||||||
|
ActorRef a = system.actorOf(Props.create(MyUntypedActor.class), "another");
|
||||||
|
ActorRef b = system.actorOf(Props.create(Follower.class, getRef()));
|
||||||
|
expectMsgEquals(a);
|
||||||
system.stop(a);
|
system.stop(a);
|
||||||
|
watch(b);
|
||||||
|
expectTerminated(b);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void usePatternsGracefulStop() throws Exception {
|
public void usePatternsGracefulStop() throws Exception {
|
||||||
ActorRef actorRef = system.actorOf(new Props(MyUntypedActor.class));
|
ActorRef actorRef = system.actorOf(Props.create(MyUntypedActor.class));
|
||||||
//#gracefulStop
|
//#gracefulStop
|
||||||
try {
|
try {
|
||||||
Future<Boolean> stopped =
|
Future<Boolean> stopped =
|
||||||
|
|
@ -223,68 +434,143 @@ public class UntypedActorDocTestBase {
|
||||||
}
|
}
|
||||||
|
|
||||||
class Result {
|
class Result {
|
||||||
final int x;
|
final String x;
|
||||||
final String s;
|
final String s;
|
||||||
|
|
||||||
public Result(int x, String s) {
|
public Result(String x, String s) {
|
||||||
this.x = x;
|
this.x = x;
|
||||||
this.s = s;
|
this.s = s;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
final int prime = 31;
|
||||||
|
int result = 1;
|
||||||
|
result = prime * result + ((s == null) ? 0 : s.hashCode());
|
||||||
|
result = prime * result + ((x == null) ? 0 : x.hashCode());
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (this == obj)
|
||||||
|
return true;
|
||||||
|
if (obj == null)
|
||||||
|
return false;
|
||||||
|
if (getClass() != obj.getClass())
|
||||||
|
return false;
|
||||||
|
Result other = (Result) obj;
|
||||||
|
if (s == null) {
|
||||||
|
if (other.s != null)
|
||||||
|
return false;
|
||||||
|
} else if (!s.equals(other.s))
|
||||||
|
return false;
|
||||||
|
if (x == null) {
|
||||||
|
if (other.x != null)
|
||||||
|
return false;
|
||||||
|
} else if (!x.equals(other.x))
|
||||||
|
return false;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void usePatternsAskPipe() {
|
public void usePatternsAskPipe() {
|
||||||
ActorRef actorA = system.actorOf(new Props(MyUntypedActor.class));
|
new JavaTestKit(system) {
|
||||||
ActorRef actorB = system.actorOf(new Props(MyUntypedActor.class));
|
{
|
||||||
ActorRef actorC = system.actorOf(new Props(MyUntypedActor.class));
|
ActorRef actorA = system.actorOf(Props.create(MyUntypedActor.class));
|
||||||
|
ActorRef actorB = system.actorOf(Props.create(MyUntypedActor.class));
|
||||||
|
ActorRef actorC = getRef();
|
||||||
|
|
||||||
//#ask-pipe
|
//#ask-pipe
|
||||||
final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS));
|
final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS));
|
||||||
|
|
||||||
final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>();
|
final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>();
|
||||||
futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout
|
futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout
|
||||||
futures.add(ask(actorB, "another request", t)); // using timeout from above
|
futures.add(ask(actorB, "another request", t)); // using timeout from
|
||||||
|
// above
|
||||||
|
|
||||||
final Future<Iterable<Object>> aggregate =
|
final Future<Iterable<Object>> aggregate = Futures.sequence(futures,
|
||||||
Futures.sequence(futures, system.dispatcher());
|
system.dispatcher());
|
||||||
|
|
||||||
final Future<Result> transformed = aggregate.map(
|
final Future<Result> transformed = aggregate.map(
|
||||||
new Mapper<Iterable<Object>, Result>() {
|
new Mapper<Iterable<Object>, Result>() {
|
||||||
public Result apply(Iterable<Object> coll) {
|
public Result apply(Iterable<Object> coll) {
|
||||||
final Iterator<Object> it = coll.iterator();
|
final Iterator<Object> it = coll.iterator();
|
||||||
|
final String x = (String) it.next();
|
||||||
final String s = (String) it.next();
|
final String s = (String) it.next();
|
||||||
final int x = (Integer) it.next();
|
|
||||||
return new Result(x, s);
|
return new Result(x, s);
|
||||||
}
|
}
|
||||||
}, system.dispatcher());
|
}, system.dispatcher());
|
||||||
|
|
||||||
pipe(transformed, system.dispatcher()).to(actorC);
|
pipe(transformed, system.dispatcher()).to(actorC);
|
||||||
//#ask-pipe
|
//#ask-pipe
|
||||||
|
|
||||||
|
expectMsgEquals(new Result("request", "another request"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
//#props-factory
|
||||||
|
public static class DemoActor extends UntypedActor {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create Props for an actor of this type.
|
||||||
|
* @param name The name to be passed to this actor’s constructor.
|
||||||
|
* @return a Props for creating this actor, which can then be further configured
|
||||||
|
* (e.g. calling `.withDispatcher()` on it)
|
||||||
|
*/
|
||||||
|
public static Props mkProps(String name) {
|
||||||
|
return Props.create(DemoActor.class, name);
|
||||||
|
}
|
||||||
|
|
||||||
|
final String name;
|
||||||
|
|
||||||
|
public DemoActor(String name) {
|
||||||
|
this.name = name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onReceive(Object msg) {
|
||||||
|
// some behavior here
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
//#props-factory
|
||||||
|
{
|
||||||
|
if (system != null)
|
||||||
|
//#props-factory
|
||||||
|
system.actorOf(DemoActor.mkProps("hello"));
|
||||||
|
//#props-factory
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class MyActor extends UntypedActor {
|
public static class MyActor extends UntypedActor {
|
||||||
|
|
||||||
|
final String s;
|
||||||
|
|
||||||
public MyActor(String s) {
|
public MyActor(String s) {
|
||||||
|
this.s = s;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void onReceive(Object message) throws Exception {
|
public void onReceive(Object message) {
|
||||||
try {
|
getSender().tell(s, getSelf());
|
||||||
operation();
|
|
||||||
} catch (Exception e) {
|
|
||||||
getSender().tell(new akka.actor.Status.Failure(e), getSelf());
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void operation() {
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This section must be kept in sync with the actual Actor trait.
|
||||||
|
*
|
||||||
|
* BOYSCOUT RULE: whenever you read this, verify that!
|
||||||
|
*/
|
||||||
//#lifecycle-callbacks
|
//#lifecycle-callbacks
|
||||||
public void preStart() {
|
public void preStart() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void preRestart(Throwable reason, Option<Object> message) {
|
public void preRestart(Throwable reason, scala.Option<Object> message) {
|
||||||
for (ActorRef each : getContext().getChildren())
|
for (ActorRef each : getContext().getChildren()) {
|
||||||
|
getContext().unwatch(each);
|
||||||
getContext().stop(each);
|
getContext().stop(each);
|
||||||
|
}
|
||||||
postStop();
|
postStop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -297,7 +583,7 @@ public class UntypedActorDocTestBase {
|
||||||
//#lifecycle-callbacks
|
//#lifecycle-callbacks
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class MyAskActor extends UntypedActor {
|
public class MyAskActor extends UntypedActor {
|
||||||
|
|
||||||
public void onReceive(Object message) throws Exception {
|
public void onReceive(Object message) throws Exception {
|
||||||
//#reply-exception
|
//#reply-exception
|
||||||
|
|
@ -409,7 +695,7 @@ public class UntypedActorDocTestBase {
|
||||||
static
|
static
|
||||||
//#identify
|
//#identify
|
||||||
public class Follower extends UntypedActor {
|
public class Follower extends UntypedActor {
|
||||||
String identifyId = "1";
|
final String identifyId = "1";
|
||||||
{
|
{
|
||||||
ActorSelection selection =
|
ActorSelection selection =
|
||||||
getContext().actorSelection("/user/another");
|
getContext().actorSelection("/user/another");
|
||||||
|
|
@ -417,6 +703,13 @@ public class UntypedActorDocTestBase {
|
||||||
}
|
}
|
||||||
ActorRef another;
|
ActorRef another;
|
||||||
|
|
||||||
|
//#test-omitted
|
||||||
|
final ActorRef probe;
|
||||||
|
public Follower(ActorRef probe) {
|
||||||
|
this.probe = probe;
|
||||||
|
}
|
||||||
|
//#test-omitted
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onReceive(Object message) {
|
public void onReceive(Object message) {
|
||||||
if (message instanceof ActorIdentity) {
|
if (message instanceof ActorIdentity) {
|
||||||
|
|
@ -428,6 +721,9 @@ public class UntypedActorDocTestBase {
|
||||||
else {
|
else {
|
||||||
another = ref;
|
another = ref;
|
||||||
getContext().watch(another);
|
getContext().watch(another);
|
||||||
|
//#test-omitted
|
||||||
|
probe.tell(ref, getSelf());
|
||||||
|
//#test-omitted
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else if (message instanceof Terminated) {
|
} else if (message instanceof Terminated) {
|
||||||
|
|
|
||||||
|
|
@ -43,7 +43,7 @@ public class UntypedActorSwapper {
|
||||||
|
|
||||||
public static void main(String... args) {
|
public static void main(String... args) {
|
||||||
ActorSystem system = ActorSystem.create("MySystem");
|
ActorSystem system = ActorSystem.create("MySystem");
|
||||||
ActorRef swap = system.actorOf(new Props(Swapper.class));
|
ActorRef swap = system.actorOf(Props.create(Swapper.class));
|
||||||
swap.tell(SWAP, null); // logs Hi
|
swap.tell(SWAP, null); // logs Hi
|
||||||
swap.tell(SWAP, null); // logs Ho
|
swap.tell(SWAP, null); // logs Ho
|
||||||
swap.tell(SWAP, null); // logs Hi
|
swap.tell(SWAP, null); // logs Hi
|
||||||
|
|
|
||||||
|
|
@ -47,8 +47,8 @@ public class FaultHandlingDocSample {
|
||||||
"akka.actor.debug.lifecycle = on");
|
"akka.actor.debug.lifecycle = on");
|
||||||
|
|
||||||
ActorSystem system = ActorSystem.create("FaultToleranceSample", config);
|
ActorSystem system = ActorSystem.create("FaultToleranceSample", config);
|
||||||
ActorRef worker = system.actorOf(new Props(Worker.class), "worker");
|
ActorRef worker = system.actorOf(Props.create(Worker.class), "worker");
|
||||||
ActorRef listener = system.actorOf(new Props(Listener.class), "listener");
|
ActorRef listener = system.actorOf(Props.create(Listener.class), "listener");
|
||||||
// start the work and listen on progress
|
// start the work and listen on progress
|
||||||
// note that the listener is used as sender of the tell,
|
// note that the listener is used as sender of the tell,
|
||||||
// i.e. it will receive replies from the worker
|
// i.e. it will receive replies from the worker
|
||||||
|
|
@ -121,7 +121,7 @@ public class FaultHandlingDocSample {
|
||||||
// about progress
|
// about progress
|
||||||
ActorRef progressListener;
|
ActorRef progressListener;
|
||||||
final ActorRef counterService = getContext().actorOf(
|
final ActorRef counterService = getContext().actorOf(
|
||||||
new Props(CounterService.class), "counter");
|
Props.create(CounterService.class), "counter");
|
||||||
final int totalCount = 51;
|
final int totalCount = 51;
|
||||||
|
|
||||||
// Stop the CounterService child if it throws ServiceUnavailable
|
// Stop the CounterService child if it throws ServiceUnavailable
|
||||||
|
|
@ -202,6 +202,7 @@ public class FaultHandlingDocSample {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class ServiceUnavailable extends RuntimeException {
|
public static class ServiceUnavailable extends RuntimeException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public ServiceUnavailable(String msg) {
|
public ServiceUnavailable(String msg) {
|
||||||
super(msg);
|
super(msg);
|
||||||
}
|
}
|
||||||
|
|
@ -271,7 +272,7 @@ public class FaultHandlingDocSample {
|
||||||
*/
|
*/
|
||||||
void initStorage() {
|
void initStorage() {
|
||||||
storage = getContext().watch(getContext().actorOf(
|
storage = getContext().watch(getContext().actorOf(
|
||||||
new Props(Storage.class), "storage"));
|
Props.create(Storage.class), "storage"));
|
||||||
// Tell the counter, if any, to use the new storage
|
// Tell the counter, if any, to use the new storage
|
||||||
if (counter != null)
|
if (counter != null)
|
||||||
counter.tell(new UseStorage(storage), getSelf());
|
counter.tell(new UseStorage(storage), getSelf());
|
||||||
|
|
@ -286,12 +287,7 @@ public class FaultHandlingDocSample {
|
||||||
counter == null) {
|
counter == null) {
|
||||||
// Reply from Storage of the initial value, now we can create the Counter
|
// Reply from Storage of the initial value, now we can create the Counter
|
||||||
final long value = ((Entry) msg).value;
|
final long value = ((Entry) msg).value;
|
||||||
counter = getContext().actorOf(new Props().withCreator(
|
counter = getContext().actorOf(Props.create(Counter.class, key, value));
|
||||||
new UntypedActorFactory() {
|
|
||||||
public Actor create() {
|
|
||||||
return new Counter(key, value);
|
|
||||||
}
|
|
||||||
}));
|
|
||||||
// Tell the counter to use current storage
|
// Tell the counter to use current storage
|
||||||
counter.tell(new UseStorage(storage), getSelf());
|
counter.tell(new UseStorage(storage), getSelf());
|
||||||
// and send the buffered backlog to the counter
|
// and send the buffered backlog to the counter
|
||||||
|
|
@ -435,6 +431,7 @@ public class FaultHandlingDocSample {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class StorageException extends RuntimeException {
|
public static class StorageException extends RuntimeException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public StorageException(String msg) {
|
public StorageException(String msg) {
|
||||||
super(msg);
|
super(msg);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -36,7 +36,7 @@ public class DurableMailboxDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void configDefinedDispatcher() {
|
public void configDefinedDispatcher() {
|
||||||
//#dispatcher-config-use
|
//#dispatcher-config-use
|
||||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class).
|
ActorRef myActor = system.actorOf(Props.create(MyUntypedActor.class).
|
||||||
withDispatcher("my-dispatcher"), "myactor");
|
withDispatcher("my-dispatcher"), "myactor");
|
||||||
//#dispatcher-config-use
|
//#dispatcher-config-use
|
||||||
myActor.tell("test", null);
|
myActor.tell("test", null);
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,6 @@ package docs.camel;
|
||||||
import akka.util.Timeout;
|
import akka.util.Timeout;
|
||||||
import scala.concurrent.Future;
|
import scala.concurrent.Future;
|
||||||
import scala.concurrent.duration.Duration;
|
import scala.concurrent.duration.Duration;
|
||||||
import scala.concurrent.duration.FiniteDuration;
|
|
||||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||||
//#CamelActivation
|
//#CamelActivation
|
||||||
|
|
||||||
|
|
@ -17,13 +16,14 @@ import org.junit.Test;
|
||||||
|
|
||||||
public class ActivationTestBase {
|
public class ActivationTestBase {
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void testActivation() {
|
public void testActivation() {
|
||||||
//#CamelActivation
|
//#CamelActivation
|
||||||
|
|
||||||
// ..
|
// ..
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
Props props = new Props(MyConsumer.class);
|
Props props = Props.create(MyConsumer.class);
|
||||||
ActorRef producer = system.actorOf(props,"myproducer");
|
ActorRef producer = system.actorOf(props,"myproducer");
|
||||||
Camel camel = CamelExtension.get(system);
|
Camel camel = CamelExtension.get(system);
|
||||||
// get a future reference to the activation of the endpoint of the Consumer Actor
|
// get a future reference to the activation of the endpoint of the Consumer Actor
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,7 @@ public class CustomRouteTestBase {
|
||||||
//#CustomRoute
|
//#CustomRoute
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
Camel camel = CamelExtension.get(system);
|
Camel camel = CamelExtension.get(system);
|
||||||
ActorRef responder = system.actorOf(new Props(Responder.class), "TestResponder");
|
ActorRef responder = system.actorOf(Props.create(Responder.class), "TestResponder");
|
||||||
camel.context().addRoutes(new CustomRouteBuilder(responder));
|
camel.context().addRoutes(new CustomRouteBuilder(responder));
|
||||||
//#CustomRoute
|
//#CustomRoute
|
||||||
system.stop(responder);
|
system.stop(responder);
|
||||||
|
|
|
||||||
|
|
@ -7,14 +7,10 @@ public class OnRouteResponseTestBase {
|
||||||
public void onRouteResponse(){
|
public void onRouteResponse(){
|
||||||
//#RouteResponse
|
//#RouteResponse
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
Props receiverProps = new Props(ResponseReceiver.class);
|
Props receiverProps = Props.create(ResponseReceiver.class);
|
||||||
final ActorRef receiver = system.actorOf(receiverProps,"responseReceiver");
|
final ActorRef receiver = system.actorOf(receiverProps,"responseReceiver");
|
||||||
UntypedActorFactory factory = new UntypedActorFactory() {
|
ActorRef forwardResponse = system.actorOf(Props.create(
|
||||||
public Actor create() {
|
Forwarder.class, "http://localhost:8080/news/akka", receiver));
|
||||||
return new Forwarder("http://localhost:8080/news/akka", receiver);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
ActorRef forwardResponse = system.actorOf(new Props(factory));
|
|
||||||
// the Forwarder sends out a request to the web page and forwards the response to
|
// the Forwarder sends out a request to the web page and forwards the response to
|
||||||
// the ResponseReceiver
|
// the ResponseReceiver
|
||||||
forwardResponse.tell("some request", null);
|
forwardResponse.tell("some request", null);
|
||||||
|
|
|
||||||
|
|
@ -14,17 +14,18 @@ public class ProducerTestBase {
|
||||||
public void tellJmsProducer() {
|
public void tellJmsProducer() {
|
||||||
//#TellProducer
|
//#TellProducer
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
Props props = new Props(Orders.class);
|
Props props = Props.create(Orders.class);
|
||||||
ActorRef producer = system.actorOf(props, "jmsproducer");
|
ActorRef producer = system.actorOf(props, "jmsproducer");
|
||||||
producer.tell("<order amount=\"100\" currency=\"PLN\" itemId=\"12345\"/>", null);
|
producer.tell("<order amount=\"100\" currency=\"PLN\" itemId=\"12345\"/>", null);
|
||||||
//#TellProducer
|
//#TellProducer
|
||||||
system.shutdown();
|
system.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
public void askProducer() {
|
public void askProducer() {
|
||||||
//#AskProducer
|
//#AskProducer
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
Props props = new Props(FirstProducer.class);
|
Props props = Props.create(FirstProducer.class);
|
||||||
ActorRef producer = system.actorOf(props,"myproducer");
|
ActorRef producer = system.actorOf(props,"myproducer");
|
||||||
Future<Object> future = Patterns.ask(producer, "some request", 1000);
|
Future<Object> future = Patterns.ask(producer, "some request", 1000);
|
||||||
//#AskProducer
|
//#AskProducer
|
||||||
|
|
@ -35,7 +36,7 @@ public class ProducerTestBase {
|
||||||
public void correlate(){
|
public void correlate(){
|
||||||
//#Correlate
|
//#Correlate
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
Props props = new Props(Orders.class);
|
Props props = Props.create(Orders.class);
|
||||||
ActorRef producer = system.actorOf(props,"jmsproducer");
|
ActorRef producer = system.actorOf(props,"jmsproducer");
|
||||||
Map<String,Object> headers = new HashMap<String, Object>();
|
Map<String,Object> headers = new HashMap<String, Object>();
|
||||||
headers.put(CamelMessage.MessageExchangeId(),"123");
|
headers.put(CamelMessage.MessageExchangeId(),"123");
|
||||||
|
|
|
||||||
|
|
@ -9,19 +9,15 @@ public class HttpSample {
|
||||||
// run the example in the MicroKernel. Just add the three lines below
|
// run the example in the MicroKernel. Just add the three lines below
|
||||||
// to your boot class.
|
// to your boot class.
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
final ActorRef httpTransformer = system.actorOf(new Props(HttpTransformer.class));
|
|
||||||
|
|
||||||
final ActorRef httpProducer = system.actorOf(new Props(new UntypedActorFactory(){
|
final ActorRef httpTransformer = system.actorOf(
|
||||||
public Actor create() {
|
Props.create(HttpTransformer.class));
|
||||||
return new HttpProducer(httpTransformer);
|
|
||||||
}
|
|
||||||
}));
|
|
||||||
|
|
||||||
ActorRef httpConsumer = system.actorOf(new Props(new UntypedActorFactory(){
|
final ActorRef httpProducer = system.actorOf(
|
||||||
public Actor create() {
|
Props.create(HttpProducer.class, httpTransformer));
|
||||||
return new HttpConsumer(httpProducer);
|
|
||||||
}
|
final ActorRef httpConsumer = system.actorOf(
|
||||||
}));
|
Props.create(HttpConsumer.class, httpProducer));
|
||||||
//#HttpExample
|
//#HttpExample
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,7 @@ import akka.actor.Props;
|
||||||
public class QuartzSample {
|
public class QuartzSample {
|
||||||
public static void main(String[] args) {
|
public static void main(String[] args) {
|
||||||
ActorSystem system = ActorSystem.create("my-quartz-system");
|
ActorSystem system = ActorSystem.create("my-quartz-system");
|
||||||
system.actorOf(new Props(MyQuartzActor.class));
|
system.actorOf(Props.create(MyQuartzActor.class));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//#QuartzExample
|
//#QuartzExample
|
||||||
|
|
@ -4,25 +4,16 @@ import akka.actor.*;
|
||||||
import akka.camel.CamelExtension;
|
import akka.camel.CamelExtension;
|
||||||
|
|
||||||
public class CustomRouteSample {
|
public class CustomRouteSample {
|
||||||
|
@SuppressWarnings("unused")
|
||||||
public static void main(String[] args) {
|
public static void main(String[] args) {
|
||||||
try {
|
try {
|
||||||
//#CustomRouteExample
|
//#CustomRouteExample
|
||||||
// the below lines can be added to a Boot class, so that you can run the
|
// the below lines can be added to a Boot class, so that you can run the
|
||||||
// example from a MicroKernel
|
// example from a MicroKernel
|
||||||
ActorSystem system = ActorSystem.create("some-system");
|
ActorSystem system = ActorSystem.create("some-system");
|
||||||
final ActorRef producer = system.actorOf(new Props(Producer1.class));
|
final ActorRef producer = system.actorOf(Props.create(Producer1.class));
|
||||||
final ActorRef mediator = system.actorOf(new Props(new UntypedActorFactory() {
|
final ActorRef mediator = system.actorOf(Props.create(Transformer.class, producer));
|
||||||
public Actor create() {
|
final ActorRef consumer = system.actorOf(Props.create(Consumer3.class, mediator));
|
||||||
return new Transformer(producer);
|
|
||||||
|
|
||||||
}
|
|
||||||
}));
|
|
||||||
ActorRef consumer = system.actorOf(new Props(new UntypedActorFactory() {
|
|
||||||
public Actor create() {
|
|
||||||
return new Consumer3(mediator);
|
|
||||||
|
|
||||||
}
|
|
||||||
}));
|
|
||||||
CamelExtension.get(system).context().addRoutes(new CustomRouteBuilder());
|
CamelExtension.get(system).context().addRoutes(new CustomRouteBuilder());
|
||||||
//#CustomRouteExample
|
//#CustomRouteExample
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
|
|
||||||
|
|
@ -58,32 +58,36 @@ public class DispatcherDocTestBase {
|
||||||
system = null;
|
system = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void defineDispatcherInConfig() {
|
public void defineDispatcherInConfig() {
|
||||||
//#defining-dispatcher-in-config
|
//#defining-dispatcher-in-config
|
||||||
ActorRef myActor =
|
ActorRef myActor =
|
||||||
system.actorOf(new Props(MyUntypedActor.class),
|
system.actorOf(Props.create(MyUntypedActor.class),
|
||||||
"myactor");
|
"myactor");
|
||||||
//#defining-dispatcher-in-config
|
//#defining-dispatcher-in-config
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void defineDispatcherInCode() {
|
public void defineDispatcherInCode() {
|
||||||
//#defining-dispatcher-in-code
|
//#defining-dispatcher-in-code
|
||||||
ActorRef myActor =
|
ActorRef myActor =
|
||||||
system.actorOf(new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"),
|
system.actorOf(Props.create(MyUntypedActor.class).withDispatcher("my-dispatcher"),
|
||||||
"myactor3");
|
"myactor3");
|
||||||
//#defining-dispatcher-in-code
|
//#defining-dispatcher-in-code
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void definePinnedDispatcher() {
|
public void definePinnedDispatcher() {
|
||||||
//#defining-pinned-dispatcher
|
//#defining-pinned-dispatcher
|
||||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class)
|
ActorRef myActor = system.actorOf(Props.create(MyUntypedActor.class)
|
||||||
.withDispatcher("my-pinned-dispatcher"));
|
.withDispatcher("my-pinned-dispatcher"));
|
||||||
//#defining-pinned-dispatcher
|
//#defining-pinned-dispatcher
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
public void compileLookup() {
|
public void compileLookup() {
|
||||||
//#lookup
|
//#lookup
|
||||||
// this is scala.concurrent.ExecutionContext
|
// this is scala.concurrent.ExecutionContext
|
||||||
|
|
@ -97,22 +101,11 @@ public class DispatcherDocTestBase {
|
||||||
JavaTestKit probe = new JavaTestKit(system);
|
JavaTestKit probe = new JavaTestKit(system);
|
||||||
//#prio-dispatcher
|
//#prio-dispatcher
|
||||||
|
|
||||||
// We create a new Actor that just prints out what it processes
|
class Demo extends UntypedActor {
|
||||||
ActorRef myActor = system.actorOf(
|
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||||
new Props().withCreator(new UntypedActorFactory() {
|
|
||||||
public UntypedActor create() {
|
|
||||||
return new UntypedActor() {
|
|
||||||
LoggingAdapter log =
|
|
||||||
Logging.getLogger(getContext().system(), this);
|
|
||||||
{
|
{
|
||||||
for(Object msg : new Object[] {
|
for (Object msg : new Object[] { "lowpriority", "lowpriority",
|
||||||
"lowpriority",
|
"highpriority", "pigdog", "pigdog2", "pigdog3", "highpriority",
|
||||||
"lowpriority",
|
|
||||||
"highpriority",
|
|
||||||
"pigdog",
|
|
||||||
"pigdog2",
|
|
||||||
"pigdog3",
|
|
||||||
"highpriority",
|
|
||||||
PoisonPill.getInstance() }) {
|
PoisonPill.getInstance() }) {
|
||||||
getSelf().tell(msg, getSelf());
|
getSelf().tell(msg, getSelf());
|
||||||
}
|
}
|
||||||
|
|
@ -121,9 +114,11 @@ public class DispatcherDocTestBase {
|
||||||
public void onReceive(Object message) {
|
public void onReceive(Object message) {
|
||||||
log.info(message.toString());
|
log.info(message.toString());
|
||||||
}
|
}
|
||||||
};
|
|
||||||
}
|
}
|
||||||
}).withDispatcher("prio-dispatcher"));
|
|
||||||
|
// We create a new Actor that just prints out what it processes
|
||||||
|
ActorRef myActor = system.actorOf(Props.create(Demo.class, this)
|
||||||
|
.withDispatcher("prio-dispatcher"));
|
||||||
|
|
||||||
/*
|
/*
|
||||||
Logs:
|
Logs:
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,6 @@ import org.junit.Test;
|
||||||
|
|
||||||
import scala.Option;
|
import scala.Option;
|
||||||
|
|
||||||
import akka.actor.UntypedActorFactory;
|
|
||||||
//#imports-deadletter
|
//#imports-deadletter
|
||||||
import akka.actor.Props;
|
import akka.actor.Props;
|
||||||
import akka.actor.ActorRef;
|
import akka.actor.ActorRef;
|
||||||
|
|
@ -36,11 +35,7 @@ public class LoggingDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void useLoggingActor() {
|
public void useLoggingActor() {
|
||||||
ActorSystem system = ActorSystem.create("MySystem");
|
ActorSystem system = ActorSystem.create("MySystem");
|
||||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
ActorRef myActor = system.actorOf(Props.create(MyActor.class, this));
|
||||||
public UntypedActor create() {
|
|
||||||
return new MyActor();
|
|
||||||
}
|
|
||||||
}));
|
|
||||||
myActor.tell("test", null);
|
myActor.tell("test", null);
|
||||||
system.shutdown();
|
system.shutdown();
|
||||||
}
|
}
|
||||||
|
|
@ -49,7 +44,7 @@ public class LoggingDocTestBase {
|
||||||
public void subscribeToDeadLetters() {
|
public void subscribeToDeadLetters() {
|
||||||
//#deadletters
|
//#deadletters
|
||||||
final ActorSystem system = ActorSystem.create("DeadLetters");
|
final ActorSystem system = ActorSystem.create("DeadLetters");
|
||||||
final ActorRef actor = system.actorOf(new Props(DeadLetterActor.class));
|
final ActorRef actor = system.actorOf(Props.create(DeadLetterActor.class));
|
||||||
system.eventStream().subscribe(actor, DeadLetter.class);
|
system.eventStream().subscribe(actor, DeadLetter.class);
|
||||||
//#deadletters
|
//#deadletters
|
||||||
system.shutdown();
|
system.shutdown();
|
||||||
|
|
|
||||||
|
|
@ -110,7 +110,7 @@ public class FutureDocTestBase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void useBlockingFromActor() throws Exception {
|
public void useBlockingFromActor() throws Exception {
|
||||||
ActorRef actor = system.actorOf(new Props(MyActor.class));
|
ActorRef actor = system.actorOf(Props.create(MyActor.class));
|
||||||
String msg = "hello";
|
String msg = "hello";
|
||||||
//#ask-blocking
|
//#ask-blocking
|
||||||
Timeout timeout = new Timeout(Duration.create(5, "seconds"));
|
Timeout timeout = new Timeout(Duration.create(5, "seconds"));
|
||||||
|
|
|
||||||
|
|
@ -13,12 +13,10 @@ import org.junit.Test;
|
||||||
|
|
||||||
import scala.concurrent.duration.Duration;
|
import scala.concurrent.duration.Duration;
|
||||||
|
|
||||||
import akka.actor.Actor;
|
|
||||||
import akka.actor.ActorRef;
|
import akka.actor.ActorRef;
|
||||||
import akka.actor.ActorSystem;
|
import akka.actor.ActorSystem;
|
||||||
import akka.actor.PoisonPill;
|
import akka.actor.PoisonPill;
|
||||||
import akka.actor.Props;
|
import akka.actor.Props;
|
||||||
import akka.actor.UntypedActorFactory;
|
|
||||||
import akka.io.AbstractPipelineContext;
|
import akka.io.AbstractPipelineContext;
|
||||||
import akka.io.PipelineFactory;
|
import akka.io.PipelineFactory;
|
||||||
import akka.io.PipelineInjector;
|
import akka.io.PipelineInjector;
|
||||||
|
|
@ -117,13 +115,10 @@ public class PipelineTest {
|
||||||
public void testTick() {
|
public void testTick() {
|
||||||
new JavaTestKit(system) {
|
new JavaTestKit(system) {
|
||||||
{
|
{
|
||||||
final ActorRef proc = system.actorOf(new Props(
|
class P extends Processor {
|
||||||
new UntypedActorFactory() {
|
public P(ActorRef cmds, ActorRef evts) throws Exception {
|
||||||
private static final long serialVersionUID = 1L;
|
super(cmds, evts);
|
||||||
|
}
|
||||||
@Override
|
|
||||||
public Actor create() throws Exception {
|
|
||||||
return new Processor(getRef(), getRef()) {
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onReceive(Object obj) throws Exception {
|
public void onReceive(Object obj) throws Exception {
|
||||||
|
|
@ -133,9 +128,10 @@ public class PipelineTest {
|
||||||
super.onReceive(obj);
|
super.onReceive(obj);
|
||||||
}
|
}
|
||||||
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
}), "processor");
|
|
||||||
|
final ActorRef proc = system.actorOf(Props.create(
|
||||||
|
P.class, this, getRef(), getRef()), "processor");
|
||||||
expectMsgClass(TickGenerator.Tick.class);
|
expectMsgClass(TickGenerator.Tick.class);
|
||||||
proc.tell(msg, null);
|
proc.tell(msg, null);
|
||||||
final ByteString encoded = expectMsgClass(ByteString.class);
|
final ByteString encoded = expectMsgClass(ByteString.class);
|
||||||
|
|
|
||||||
|
|
@ -66,6 +66,7 @@ public class ConsistentHashingRouterDocTestBase {
|
||||||
static
|
static
|
||||||
//#cache-actor
|
//#cache-actor
|
||||||
public final class Evict implements Serializable {
|
public final class Evict implements Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public final String key;
|
public final String key;
|
||||||
public Evict(String key) {
|
public Evict(String key) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
|
|
@ -76,6 +77,7 @@ public class ConsistentHashingRouterDocTestBase {
|
||||||
static
|
static
|
||||||
//#cache-actor
|
//#cache-actor
|
||||||
public final class Get implements Serializable, ConsistentHashable {
|
public final class Get implements Serializable, ConsistentHashable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public final String key;
|
public final String key;
|
||||||
public Get(String key) {
|
public Get(String key) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
|
|
@ -89,6 +91,7 @@ public class ConsistentHashingRouterDocTestBase {
|
||||||
static
|
static
|
||||||
//#cache-actor
|
//#cache-actor
|
||||||
public final class Entry implements Serializable {
|
public final class Entry implements Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public final String key;
|
public final String key;
|
||||||
public final String value;
|
public final String value;
|
||||||
public Entry(String key, String value) {
|
public Entry(String key, String value) {
|
||||||
|
|
@ -122,7 +125,7 @@ public class ConsistentHashingRouterDocTestBase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
ActorRef cache = system.actorOf(new Props(Cache.class).withRouter(
|
ActorRef cache = system.actorOf(Props.create(Cache.class).withRouter(
|
||||||
new ConsistentHashingRouter(10).withHashMapper(hashMapper)),
|
new ConsistentHashingRouter(10).withHashMapper(hashMapper)),
|
||||||
"cache");
|
"cache");
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -57,7 +57,7 @@ public class CustomRouterDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateDispatchers() {
|
public void demonstrateDispatchers() {
|
||||||
//#dispatchers
|
//#dispatchers
|
||||||
final ActorRef router = system.actorOf(new Props(MyActor.class)
|
final ActorRef router = system.actorOf(Props.create(MyActor.class)
|
||||||
// “head” router will run on "head" dispatcher
|
// “head” router will run on "head" dispatcher
|
||||||
.withRouter(new RoundRobinRouter(5).withDispatcher("head"))
|
.withRouter(new RoundRobinRouter(5).withDispatcher("head"))
|
||||||
// MyActor “workers” will run on "workers" dispatcher
|
// MyActor “workers” will run on "workers" dispatcher
|
||||||
|
|
@ -71,7 +71,7 @@ public class CustomRouterDocTestBase {
|
||||||
final SupervisorStrategy strategy =
|
final SupervisorStrategy strategy =
|
||||||
new OneForOneStrategy(5, Duration.create("1 minute"),
|
new OneForOneStrategy(5, Duration.create("1 minute"),
|
||||||
Collections.<Class<? extends Throwable>>singletonList(Exception.class));
|
Collections.<Class<? extends Throwable>>singletonList(Exception.class));
|
||||||
final ActorRef router = system.actorOf(new Props(MyActor.class)
|
final ActorRef router = system.actorOf(Props.create(MyActor.class)
|
||||||
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
|
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
|
||||||
//#supervision
|
//#supervision
|
||||||
}
|
}
|
||||||
|
|
@ -80,7 +80,7 @@ public class CustomRouterDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void countVotesAsIntendedNotAsInFlorida() throws Exception {
|
public void countVotesAsIntendedNotAsInFlorida() throws Exception {
|
||||||
ActorRef routedActor = system.actorOf(
|
ActorRef routedActor = system.actorOf(
|
||||||
new Props().withRouter(new VoteCountRouter()));
|
Props.empty().withRouter(new VoteCountRouter()));
|
||||||
routedActor.tell(DemocratVote, null);
|
routedActor.tell(DemocratVote, null);
|
||||||
routedActor.tell(DemocratVote, null);
|
routedActor.tell(DemocratVote, null);
|
||||||
routedActor.tell(RepublicanVote, null);
|
routedActor.tell(RepublicanVote, null);
|
||||||
|
|
@ -167,9 +167,9 @@ public class CustomRouterDocTestBase {
|
||||||
@Override
|
@Override
|
||||||
public CustomRoute createCustomRoute(RouteeProvider routeeProvider) {
|
public CustomRoute createCustomRoute(RouteeProvider routeeProvider) {
|
||||||
final ActorRef democratActor =
|
final ActorRef democratActor =
|
||||||
routeeProvider.context().actorOf(new Props(DemocratActor.class), "d");
|
routeeProvider.context().actorOf(Props.create(DemocratActor.class), "d");
|
||||||
final ActorRef republicanActor =
|
final ActorRef republicanActor =
|
||||||
routeeProvider.context().actorOf(new Props(RepublicanActor.class), "r");
|
routeeProvider.context().actorOf(Props.create(RepublicanActor.class), "r");
|
||||||
List<ActorRef> routees =
|
List<ActorRef> routees =
|
||||||
Arrays.asList(new ActorRef[] { democratActor, republicanActor });
|
Arrays.asList(new ActorRef[] { democratActor, republicanActor });
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,7 @@ public class ParentActor extends UntypedActor {
|
||||||
if (msg.equals("rrr")) {
|
if (msg.equals("rrr")) {
|
||||||
//#roundRobinRouter
|
//#roundRobinRouter
|
||||||
ActorRef roundRobinRouter = getContext().actorOf(
|
ActorRef roundRobinRouter = getContext().actorOf(
|
||||||
new Props(PrintlnActor.class).withRouter(new RoundRobinRouter(5)),
|
Props.create(PrintlnActor.class).withRouter(new RoundRobinRouter(5)),
|
||||||
"router");
|
"router");
|
||||||
for (int i = 1; i <= 10; i++) {
|
for (int i = 1; i <= 10; i++) {
|
||||||
roundRobinRouter.tell(i, getSelf());
|
roundRobinRouter.tell(i, getSelf());
|
||||||
|
|
@ -31,7 +31,7 @@ public class ParentActor extends UntypedActor {
|
||||||
} else if (msg.equals("rr")) {
|
} else if (msg.equals("rr")) {
|
||||||
//#randomRouter
|
//#randomRouter
|
||||||
ActorRef randomRouter = getContext().actorOf(
|
ActorRef randomRouter = getContext().actorOf(
|
||||||
new Props(PrintlnActor.class).withRouter(new RandomRouter(5)),
|
Props.create(PrintlnActor.class).withRouter(new RandomRouter(5)),
|
||||||
"router");
|
"router");
|
||||||
for (int i = 1; i <= 10; i++) {
|
for (int i = 1; i <= 10; i++) {
|
||||||
randomRouter.tell(i, getSelf());
|
randomRouter.tell(i, getSelf());
|
||||||
|
|
@ -40,7 +40,7 @@ public class ParentActor extends UntypedActor {
|
||||||
} else if (msg.equals("smr")) {
|
} else if (msg.equals("smr")) {
|
||||||
//#smallestMailboxRouter
|
//#smallestMailboxRouter
|
||||||
ActorRef smallestMailboxRouter = getContext().actorOf(
|
ActorRef smallestMailboxRouter = getContext().actorOf(
|
||||||
new Props(PrintlnActor.class).withRouter(new SmallestMailboxRouter(5)),
|
Props.create(PrintlnActor.class).withRouter(new SmallestMailboxRouter(5)),
|
||||||
"router");
|
"router");
|
||||||
for (int i = 1; i <= 10; i++) {
|
for (int i = 1; i <= 10; i++) {
|
||||||
smallestMailboxRouter.tell(i, getSelf());
|
smallestMailboxRouter.tell(i, getSelf());
|
||||||
|
|
@ -49,13 +49,13 @@ public class ParentActor extends UntypedActor {
|
||||||
} else if (msg.equals("br")) {
|
} else if (msg.equals("br")) {
|
||||||
//#broadcastRouter
|
//#broadcastRouter
|
||||||
ActorRef broadcastRouter = getContext().actorOf(
|
ActorRef broadcastRouter = getContext().actorOf(
|
||||||
new Props(PrintlnActor.class).withRouter(new BroadcastRouter(5)), "router");
|
Props.create(PrintlnActor.class).withRouter(new BroadcastRouter(5)), "router");
|
||||||
broadcastRouter.tell("this is a broadcast message", getSelf());
|
broadcastRouter.tell("this is a broadcast message", getSelf());
|
||||||
//#broadcastRouter
|
//#broadcastRouter
|
||||||
} else if (msg.equals("sgfcr")) {
|
} else if (msg.equals("sgfcr")) {
|
||||||
//#scatterGatherFirstCompletedRouter
|
//#scatterGatherFirstCompletedRouter
|
||||||
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
|
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
|
||||||
new Props(FibonacciActor.class).withRouter(
|
Props.create(FibonacciActor.class).withRouter(
|
||||||
new ScatterGatherFirstCompletedRouter(5, Duration.create(2, "seconds"))),
|
new ScatterGatherFirstCompletedRouter(5, Duration.create(2, "seconds"))),
|
||||||
"router");
|
"router");
|
||||||
Timeout timeout = new Timeout(Duration.create(5, "seconds"));
|
Timeout timeout = new Timeout(Duration.create(5, "seconds"));
|
||||||
|
|
|
||||||
|
|
@ -44,7 +44,7 @@ public class RouterViaConfigExample {
|
||||||
ActorSystem system = ActorSystem.create("Example", config);
|
ActorSystem system = ActorSystem.create("Example", config);
|
||||||
//#configurableRouting
|
//#configurableRouting
|
||||||
ActorRef router = system.actorOf(
|
ActorRef router = system.actorOf(
|
||||||
new Props(ExampleActor.class).withRouter(new FromConfig()), "myrouter1");
|
Props.create(ExampleActor.class).withRouter(new FromConfig()), "myrouter1");
|
||||||
//#configurableRouting
|
//#configurableRouting
|
||||||
for (int i = 1; i <= 10; i++) {
|
for (int i = 1; i <= 10; i++) {
|
||||||
router.tell(new ExampleActor.Message(i), null);
|
router.tell(new ExampleActor.Message(i), null);
|
||||||
|
|
@ -52,7 +52,7 @@ public class RouterViaConfigExample {
|
||||||
|
|
||||||
//#configurableRoutingWithResizer
|
//#configurableRoutingWithResizer
|
||||||
ActorRef router2 = system.actorOf(
|
ActorRef router2 = system.actorOf(
|
||||||
new Props(ExampleActor.class).withRouter(new FromConfig()), "myrouter2");
|
Props.create(ExampleActor.class).withRouter(new FromConfig()), "myrouter2");
|
||||||
//#configurableRoutingWithResizer
|
//#configurableRoutingWithResizer
|
||||||
for (int i = 1; i <= 10; i++) {
|
for (int i = 1; i <= 10; i++) {
|
||||||
router2.tell(new ExampleActor.Message(i), null);
|
router2.tell(new ExampleActor.Message(i), null);
|
||||||
|
|
|
||||||
|
|
@ -3,21 +3,23 @@
|
||||||
*/
|
*/
|
||||||
package docs.jrouting;
|
package docs.jrouting;
|
||||||
|
|
||||||
import akka.actor.*;
|
import java.util.Arrays;
|
||||||
import akka.remote.routing.RemoteRouterConfig;
|
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import akka.actor.ActorRef;
|
||||||
|
import akka.actor.ActorSystem;
|
||||||
|
import akka.actor.Kill;
|
||||||
|
import akka.actor.PoisonPill;
|
||||||
|
import akka.actor.Props;
|
||||||
|
import akka.actor.Terminated;
|
||||||
import akka.routing.Broadcast;
|
import akka.routing.Broadcast;
|
||||||
import akka.routing.RoundRobinRouter;
|
import akka.routing.RoundRobinRouter;
|
||||||
import akka.testkit.JavaTestKit;
|
import akka.testkit.JavaTestKit;
|
||||||
import docs.jrouting.RouterViaProgramExample.ExampleActor;
|
import docs.jrouting.RouterViaProgramExample.ExampleActor;
|
||||||
import docs.routing.RouterViaProgramDocSpec.Echo;
|
import docs.routing.RouterViaProgramDocSpec.Echo;
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
import org.junit.AfterClass;
|
|
||||||
import org.junit.Assert;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.Test;
|
|
||||||
|
|
||||||
public class RouterViaProgramDocTestBase {
|
public class RouterViaProgramDocTestBase {
|
||||||
|
|
||||||
|
|
@ -45,17 +47,18 @@ public class RouterViaProgramDocTestBase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateRouteesFromPaths() {
|
public void demonstrateRouteesFromPaths() {
|
||||||
new JavaTestKit(system) {{
|
new JavaTestKit(system) {{
|
||||||
//#programmaticRoutingRouteePaths
|
//#programmaticRoutingRouteePaths
|
||||||
ActorRef actor1 = system.actorOf(new Props(ExampleActor.class), "actor1");
|
ActorRef actor1 = system.actorOf(Props.create(ExampleActor.class), "actor1");
|
||||||
ActorRef actor2 = system.actorOf(new Props(ExampleActor.class), "actor2");
|
ActorRef actor2 = system.actorOf(Props.create(ExampleActor.class), "actor2");
|
||||||
ActorRef actor3 = system.actorOf(new Props(ExampleActor.class), "actor3");
|
ActorRef actor3 = system.actorOf(Props.create(ExampleActor.class), "actor3");
|
||||||
Iterable<String> routees = Arrays.asList(
|
Iterable<String> routees = Arrays.asList(
|
||||||
new String[] { "/user/actor1", "/user/actor2", "/user/actor3" });
|
new String[] { "/user/actor1", "/user/actor2", "/user/actor3" });
|
||||||
ActorRef router = system.actorOf(
|
ActorRef router = system.actorOf(
|
||||||
new Props().withRouter(new RoundRobinRouter(routees)));
|
Props.empty().withRouter(new RoundRobinRouter(routees)));
|
||||||
//#programmaticRoutingRouteePaths
|
//#programmaticRoutingRouteePaths
|
||||||
for (int i = 1; i <= 6; i++) {
|
for (int i = 1; i <= 6; i++) {
|
||||||
router.tell(new ExampleActor.Message(i), null);
|
router.tell(new ExampleActor.Message(i), null);
|
||||||
|
|
@ -66,7 +69,7 @@ public class RouterViaProgramDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateBroadcast() {
|
public void demonstrateBroadcast() {
|
||||||
new JavaTestKitWithSelf(system) {{
|
new JavaTestKitWithSelf(system) {{
|
||||||
ActorRef router = system.actorOf(new Props(Echo.class).withRouter(new RoundRobinRouter(5)));
|
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||||
//#broadcastDavyJonesWarning
|
//#broadcastDavyJonesWarning
|
||||||
router.tell(new Broadcast("Watch out for Davy Jones' locker"), getSelf());
|
router.tell(new Broadcast("Watch out for Davy Jones' locker"), getSelf());
|
||||||
//#broadcastDavyJonesWarning
|
//#broadcastDavyJonesWarning
|
||||||
|
|
@ -77,7 +80,7 @@ public class RouterViaProgramDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void demonstratePoisonPill() {
|
public void demonstratePoisonPill() {
|
||||||
new JavaTestKitWithSelf(system) {{
|
new JavaTestKitWithSelf(system) {{
|
||||||
ActorRef router = system.actorOf(new Props(Echo.class).withRouter(new RoundRobinRouter(5)));
|
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||||
watch(router);
|
watch(router);
|
||||||
//#poisonPill
|
//#poisonPill
|
||||||
router.tell(PoisonPill.getInstance(), getSelf());
|
router.tell(PoisonPill.getInstance(), getSelf());
|
||||||
|
|
@ -89,7 +92,7 @@ public class RouterViaProgramDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateBroadcastOfPoisonPill() {
|
public void demonstrateBroadcastOfPoisonPill() {
|
||||||
new JavaTestKitWithSelf(system) {{
|
new JavaTestKitWithSelf(system) {{
|
||||||
ActorRef router = system.actorOf(new Props(Echo.class).withRouter(new RoundRobinRouter(5)));
|
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||||
watch(router);
|
watch(router);
|
||||||
//#broadcastPoisonPill
|
//#broadcastPoisonPill
|
||||||
router.tell(new Broadcast(PoisonPill.getInstance()), getSelf());
|
router.tell(new Broadcast(PoisonPill.getInstance()), getSelf());
|
||||||
|
|
@ -101,7 +104,7 @@ public class RouterViaProgramDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateKill() {
|
public void demonstrateKill() {
|
||||||
new JavaTestKitWithSelf(system) {{
|
new JavaTestKitWithSelf(system) {{
|
||||||
ActorRef router = system.actorOf(new Props(Echo.class).withRouter(new RoundRobinRouter(5)));
|
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||||
watch(router);
|
watch(router);
|
||||||
//#kill
|
//#kill
|
||||||
router.tell(Kill.getInstance(), getSelf());
|
router.tell(Kill.getInstance(), getSelf());
|
||||||
|
|
@ -113,7 +116,7 @@ public class RouterViaProgramDocTestBase {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateBroadcastOfKill() {
|
public void demonstrateBroadcastOfKill() {
|
||||||
new JavaTestKitWithSelf(system) {{
|
new JavaTestKitWithSelf(system) {{
|
||||||
ActorRef router = system.actorOf(new Props(Echo.class).withRouter(new RoundRobinRouter(5)));
|
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||||
watch(router);
|
watch(router);
|
||||||
//#broadcastKill
|
//#broadcastKill
|
||||||
router.tell(new Broadcast(Kill.getInstance()), getSelf());
|
router.tell(new Broadcast(Kill.getInstance()), getSelf());
|
||||||
|
|
|
||||||
|
|
@ -41,25 +41,26 @@ public class RouterViaProgramExample {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
public static void main(String... args) {
|
public static void main(String... args) {
|
||||||
ActorSystem system = ActorSystem.create("RPE");
|
ActorSystem system = ActorSystem.create("RPE");
|
||||||
//#programmaticRoutingNrOfInstances
|
//#programmaticRoutingNrOfInstances
|
||||||
int nrOfInstances = 5;
|
int nrOfInstances = 5;
|
||||||
ActorRef router1 = system.actorOf(
|
ActorRef router1 = system.actorOf(
|
||||||
new Props(ExampleActor.class).withRouter(new RoundRobinRouter(nrOfInstances)));
|
Props.create(ExampleActor.class).withRouter(new RoundRobinRouter(nrOfInstances)));
|
||||||
//#programmaticRoutingNrOfInstances
|
//#programmaticRoutingNrOfInstances
|
||||||
for (int i = 1; i <= 6; i++) {
|
for (int i = 1; i <= 6; i++) {
|
||||||
router1.tell(new ExampleActor.Message(i), null);
|
router1.tell(new ExampleActor.Message(i), null);
|
||||||
}
|
}
|
||||||
|
|
||||||
//#programmaticRoutingRoutees
|
//#programmaticRoutingRoutees
|
||||||
ActorRef actor1 = system.actorOf(new Props(ExampleActor.class));
|
ActorRef actor1 = system.actorOf(Props.create(ExampleActor.class));
|
||||||
ActorRef actor2 = system.actorOf(new Props(ExampleActor.class));
|
ActorRef actor2 = system.actorOf(Props.create(ExampleActor.class));
|
||||||
ActorRef actor3 = system.actorOf(new Props(ExampleActor.class));
|
ActorRef actor3 = system.actorOf(Props.create(ExampleActor.class));
|
||||||
Iterable<ActorRef> routees = Arrays.asList(
|
Iterable<ActorRef> routees = Arrays.asList(
|
||||||
new ActorRef[] { actor1, actor2, actor3 });
|
new ActorRef[] { actor1, actor2, actor3 });
|
||||||
ActorRef router2 = system.actorOf(
|
ActorRef router2 = system.actorOf(
|
||||||
new Props().withRouter(RoundRobinRouter.create(routees)));
|
Props.empty().withRouter(RoundRobinRouter.create(routees)));
|
||||||
//#programmaticRoutingRoutees
|
//#programmaticRoutingRoutees
|
||||||
for (int i = 1; i <= 6; i++) {
|
for (int i = 1; i <= 6; i++) {
|
||||||
router2.tell(new ExampleActor.Message(i), null);
|
router2.tell(new ExampleActor.Message(i), null);
|
||||||
|
|
@ -70,7 +71,7 @@ public class RouterViaProgramExample {
|
||||||
int upperBound = 15;
|
int upperBound = 15;
|
||||||
DefaultResizer resizer = new DefaultResizer(lowerBound, upperBound);
|
DefaultResizer resizer = new DefaultResizer(lowerBound, upperBound);
|
||||||
ActorRef router3 = system.actorOf(
|
ActorRef router3 = system.actorOf(
|
||||||
new Props(ExampleActor.class).withRouter(new RoundRobinRouter(resizer)));
|
Props.create(ExampleActor.class).withRouter(new RoundRobinRouter(resizer)));
|
||||||
//#programmaticRoutingWithResizer
|
//#programmaticRoutingWithResizer
|
||||||
for (int i = 1; i <= 6; i++) {
|
for (int i = 1; i <= 6; i++) {
|
||||||
router3.tell(new ExampleActor.Message(i), null);
|
router3.tell(new ExampleActor.Message(i), null);
|
||||||
|
|
@ -80,11 +81,12 @@ public class RouterViaProgramExample {
|
||||||
Address addr1 = new Address("akka", "remotesys", "otherhost", 1234);
|
Address addr1 = new Address("akka", "remotesys", "otherhost", 1234);
|
||||||
Address addr2 = AddressFromURIString.parse("akka://othersys@anotherhost:1234");
|
Address addr2 = AddressFromURIString.parse("akka://othersys@anotherhost:1234");
|
||||||
Address[] addresses = new Address[] { addr1, addr2 };
|
Address[] addresses = new Address[] { addr1, addr2 };
|
||||||
ActorRef routerRemote = system.actorOf(new Props(ExampleActor.class)
|
ActorRef routerRemote = system.actorOf(Props.create(ExampleActor.class)
|
||||||
.withRouter(new RemoteRouterConfig(new RoundRobinRouter(5), addresses)));
|
.withRouter(new RemoteRouterConfig(new RoundRobinRouter(5), addresses)));
|
||||||
//#remoteRoutees
|
//#remoteRoutees
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
private class CompileCheckJavaDocsForRouting extends UntypedActor {
|
private class CompileCheckJavaDocsForRouting extends UntypedActor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -121,13 +121,7 @@ public class SchedulerPatternTest {
|
||||||
public void scheduleInConstructor() {
|
public void scheduleInConstructor() {
|
||||||
new TestSchedule(system) {{
|
new TestSchedule(system) {{
|
||||||
final JavaTestKit probe = new JavaTestKit(system);
|
final JavaTestKit probe = new JavaTestKit(system);
|
||||||
|
final Props props = Props.create(ScheduleInConstructor.class, probe.getRef());
|
||||||
final Props props = new Props(new UntypedActorFactory() {
|
|
||||||
public UntypedActor create() {
|
|
||||||
return new ScheduleInConstructor(probe.getRef());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
testSchedule(probe, props, duration("3000 millis"), duration("2000 millis"));
|
testSchedule(probe, props, duration("3000 millis"), duration("2000 millis"));
|
||||||
}};
|
}};
|
||||||
}
|
}
|
||||||
|
|
@ -135,16 +129,9 @@ public class SchedulerPatternTest {
|
||||||
@Test
|
@Test
|
||||||
@Ignore // no way to tag this as timing sensitive
|
@Ignore // no way to tag this as timing sensitive
|
||||||
public void scheduleInReceive() {
|
public void scheduleInReceive() {
|
||||||
|
|
||||||
new TestSchedule(system) {{
|
new TestSchedule(system) {{
|
||||||
final JavaTestKit probe = new JavaTestKit(system);
|
final JavaTestKit probe = new JavaTestKit(system);
|
||||||
|
final Props props = Props.create(ScheduleInReceive.class, probe.getRef());
|
||||||
final Props props = new Props(new UntypedActorFactory() {
|
|
||||||
public UntypedActor create() {
|
|
||||||
return new ScheduleInReceive(probe.getRef());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
testSchedule(probe, props, duration("3000 millis"), duration("2500 millis"));
|
testSchedule(probe, props, duration("3000 millis"), duration("2500 millis"));
|
||||||
}};
|
}};
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -45,7 +45,7 @@ public class SupervisedAsk {
|
||||||
public void onReceive(Object message) throws Exception {
|
public void onReceive(Object message) throws Exception {
|
||||||
if (message instanceof AskParam) {
|
if (message instanceof AskParam) {
|
||||||
ActorRef supervisor = getContext().actorOf(
|
ActorRef supervisor = getContext().actorOf(
|
||||||
Props.apply(AskSupervisor.class));
|
Props.create(AskSupervisor.class));
|
||||||
supervisor.forward(message, getContext());
|
supervisor.forward(message, getContext());
|
||||||
} else {
|
} else {
|
||||||
unhandled(message);
|
unhandled(message);
|
||||||
|
|
@ -104,6 +104,6 @@ public class SupervisedAsk {
|
||||||
|
|
||||||
synchronized public static ActorRef createSupervisorCreator(
|
synchronized public static ActorRef createSupervisorCreator(
|
||||||
ActorRefFactory factory) {
|
ActorRefFactory factory) {
|
||||||
return factory.actorOf(Props.apply(AskSupervisorCreator.class));
|
return factory.actorOf(Props.create(AskSupervisorCreator.class));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -18,7 +18,7 @@ public class SupervisedAskSpec {
|
||||||
ActorRef supervisorCreator = SupervisedAsk
|
ActorRef supervisorCreator = SupervisedAsk
|
||||||
.createSupervisorCreator(actorSystem);
|
.createSupervisorCreator(actorSystem);
|
||||||
Future<Object> finished = SupervisedAsk.askOf(supervisorCreator,
|
Future<Object> finished = SupervisedAsk.askOf(supervisorCreator,
|
||||||
Props.apply(someActor), message, timeout);
|
Props.create(someActor), message, timeout);
|
||||||
return Await.result(finished, timeout.duration());
|
return Await.result(finished, timeout.duration());
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// exception propagated by supervision
|
// exception propagated by supervision
|
||||||
|
|
|
||||||
|
|
@ -48,7 +48,7 @@ public class RemoteDeploymentDocTestBase {
|
||||||
addr = AddressFromURIString.parse("akka.tcp://sys@host:1234"); // the same
|
addr = AddressFromURIString.parse("akka.tcp://sys@host:1234"); // the same
|
||||||
//#make-address
|
//#make-address
|
||||||
//#deploy
|
//#deploy
|
||||||
ActorRef ref = system.actorOf(new Props(SampleActor.class).withDeploy(
|
ActorRef ref = system.actorOf(Props.create(SampleActor.class).withDeploy(
|
||||||
new Deploy(new RemoteScope(addr))));
|
new Deploy(new RemoteScope(addr))));
|
||||||
//#deploy
|
//#deploy
|
||||||
assert ref.path().address().equals(addr);
|
assert ref.path().address().equals(addr);
|
||||||
|
|
@ -58,7 +58,7 @@ public class RemoteDeploymentDocTestBase {
|
||||||
public void demonstrateSampleActor() {
|
public void demonstrateSampleActor() {
|
||||||
//#sample-actor
|
//#sample-actor
|
||||||
|
|
||||||
ActorRef actor = system.actorOf(new Props(SampleActor.class), "sampleActor");
|
ActorRef actor = system.actorOf(Props.create(SampleActor.class), "sampleActor");
|
||||||
actor.tell("Pretty slick", null);
|
actor.tell("Pretty slick", null);
|
||||||
//#sample-actor
|
//#sample-actor
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,7 +20,6 @@ import akka.actor.PoisonPill;
|
||||||
import akka.actor.Props;
|
import akka.actor.Props;
|
||||||
import akka.actor.Terminated;
|
import akka.actor.Terminated;
|
||||||
import akka.actor.UntypedActor;
|
import akka.actor.UntypedActor;
|
||||||
import akka.actor.UntypedActorFactory;
|
|
||||||
import scala.concurrent.Await;
|
import scala.concurrent.Await;
|
||||||
import scala.concurrent.Future;
|
import scala.concurrent.Future;
|
||||||
import akka.testkit.CallingThreadDispatcher;
|
import akka.testkit.CallingThreadDispatcher;
|
||||||
|
|
@ -63,7 +62,7 @@ public class TestKitDocTest {
|
||||||
//#test-actor-ref
|
//#test-actor-ref
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateTestActorRef() {
|
public void demonstrateTestActorRef() {
|
||||||
final Props props = new Props(MyActor.class);
|
final Props props = Props.create(MyActor.class);
|
||||||
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testA");
|
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testA");
|
||||||
final MyActor actor = ref.underlyingActor();
|
final MyActor actor = ref.underlyingActor();
|
||||||
assertTrue(actor.testMe());
|
assertTrue(actor.testMe());
|
||||||
|
|
@ -73,7 +72,7 @@ public class TestKitDocTest {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateAsk() throws Exception {
|
public void demonstrateAsk() throws Exception {
|
||||||
//#test-behavior
|
//#test-behavior
|
||||||
final Props props = new Props(MyActor.class);
|
final Props props = Props.create(MyActor.class);
|
||||||
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testB");
|
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testB");
|
||||||
final Future<Object> future = akka.pattern.Patterns.ask(ref, "say42", 3000);
|
final Future<Object> future = akka.pattern.Patterns.ask(ref, "say42", 3000);
|
||||||
assertTrue(future.isCompleted());
|
assertTrue(future.isCompleted());
|
||||||
|
|
@ -84,7 +83,7 @@ public class TestKitDocTest {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateExceptions() {
|
public void demonstrateExceptions() {
|
||||||
//#test-expecting-exceptions
|
//#test-expecting-exceptions
|
||||||
final Props props = new Props(MyActor.class);
|
final Props props = Props.create(MyActor.class);
|
||||||
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "myActor");
|
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "myActor");
|
||||||
try {
|
try {
|
||||||
ref.receive(new Exception("expected"));
|
ref.receive(new Exception("expected"));
|
||||||
|
|
@ -208,7 +207,7 @@ public class TestKitDocTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@SuppressWarnings("unchecked") // due to generic varargs
|
@SuppressWarnings({ "unchecked", "unused" }) // due to generic varargs
|
||||||
public void demonstrateExpect() {
|
public void demonstrateExpect() {
|
||||||
new JavaTestKit(system) {{
|
new JavaTestKit(system) {{
|
||||||
getRef().tell("hello", null);
|
getRef().tell("hello", null);
|
||||||
|
|
@ -273,9 +272,11 @@ public class TestKitDocTest {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateProbe() {
|
public void demonstrateProbe() {
|
||||||
//#test-probe
|
//#test-probe
|
||||||
|
new JavaTestKit(system) {{
|
||||||
// simple actor which just forwards messages
|
// simple actor which just forwards messages
|
||||||
class Forwarder extends UntypedActor {
|
class Forwarder extends UntypedActor {
|
||||||
final ActorRef target;
|
final ActorRef target;
|
||||||
|
@SuppressWarnings("unused")
|
||||||
public Forwarder(ActorRef target) {
|
public Forwarder(ActorRef target) {
|
||||||
this.target = target;
|
this.target = target;
|
||||||
}
|
}
|
||||||
|
|
@ -284,17 +285,11 @@ public class TestKitDocTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
new JavaTestKit(system) {{
|
|
||||||
// create a test probe
|
// create a test probe
|
||||||
final JavaTestKit probe = new JavaTestKit(system);
|
final JavaTestKit probe = new JavaTestKit(system);
|
||||||
|
|
||||||
// create a forwarder, injecting the probe’s testActor
|
// create a forwarder, injecting the probe’s testActor
|
||||||
final Props props = new Props(new UntypedActorFactory() {
|
final Props props = Props.create(Forwarder.class, this, probe.getRef());
|
||||||
private static final long serialVersionUID = 8927158735963950216L;
|
|
||||||
public UntypedActor create() {
|
|
||||||
return new Forwarder(probe.getRef());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
final ActorRef forwarder = system.actorOf(props, "forwarder");
|
final ActorRef forwarder = system.actorOf(props, "forwarder");
|
||||||
|
|
||||||
// verify correct forwarding
|
// verify correct forwarding
|
||||||
|
|
@ -327,7 +322,7 @@ public class TestKitDocTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateWatch() {
|
public void demonstrateWatch() {
|
||||||
final ActorRef target = system.actorOf(new Props(MyActor.class));
|
final ActorRef target = system.actorOf(Props.create(MyActor.class));
|
||||||
//#test-probe-watch
|
//#test-probe-watch
|
||||||
new JavaTestKit(system) {{
|
new JavaTestKit(system) {{
|
||||||
final JavaTestKit probe = new JavaTestKit(system);
|
final JavaTestKit probe = new JavaTestKit(system);
|
||||||
|
|
@ -411,7 +406,7 @@ public class TestKitDocTest {
|
||||||
public void demonstrateCTD() {
|
public void demonstrateCTD() {
|
||||||
//#calling-thread-dispatcher
|
//#calling-thread-dispatcher
|
||||||
system.actorOf(
|
system.actorOf(
|
||||||
new Props(MyActor.class)
|
Props.create(MyActor.class)
|
||||||
.withDispatcher(CallingThreadDispatcher.Id()));
|
.withDispatcher(CallingThreadDispatcher.Id()));
|
||||||
//#calling-thread-dispatcher
|
//#calling-thread-dispatcher
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -53,7 +53,7 @@ public class TestKitSampleTest {
|
||||||
* if you want to receive actor replies or use Within(), etc.
|
* if you want to receive actor replies or use Within(), etc.
|
||||||
*/
|
*/
|
||||||
new JavaTestKit(system) {{
|
new JavaTestKit(system) {{
|
||||||
final Props props = new Props(SomeActor.class);
|
final Props props = Props.create(SomeActor.class);
|
||||||
final ActorRef subject = system.actorOf(props);
|
final ActorRef subject = system.actorOf(props);
|
||||||
|
|
||||||
// can also use JavaTestKit “from the outside”
|
// can also use JavaTestKit “from the outside”
|
||||||
|
|
|
||||||
|
|
@ -23,8 +23,8 @@ public class TransactorDocTest {
|
||||||
//#coordinated-example
|
//#coordinated-example
|
||||||
ActorSystem system = ActorSystem.create("CoordinatedExample");
|
ActorSystem system = ActorSystem.create("CoordinatedExample");
|
||||||
|
|
||||||
ActorRef counter1 = system.actorOf(new Props(CoordinatedCounter.class));
|
ActorRef counter1 = system.actorOf(Props.create(CoordinatedCounter.class));
|
||||||
ActorRef counter2 = system.actorOf(new Props(CoordinatedCounter.class));
|
ActorRef counter2 = system.actorOf(Props.create(CoordinatedCounter.class));
|
||||||
|
|
||||||
Timeout timeout = new Timeout(5, SECONDS);
|
Timeout timeout = new Timeout(5, SECONDS);
|
||||||
|
|
||||||
|
|
@ -47,7 +47,7 @@ public class TransactorDocTest {
|
||||||
//#create-coordinated
|
//#create-coordinated
|
||||||
|
|
||||||
ActorSystem system = ActorSystem.create("CoordinatedApi");
|
ActorSystem system = ActorSystem.create("CoordinatedApi");
|
||||||
ActorRef actor = system.actorOf(new Props(Coordinator.class));
|
ActorRef actor = system.actorOf(Props.create(Coordinator.class));
|
||||||
|
|
||||||
//#send-coordinated
|
//#send-coordinated
|
||||||
actor.tell(new Coordinated(new Message(), timeout), null);
|
actor.tell(new Coordinated(new Message(), timeout), null);
|
||||||
|
|
@ -65,7 +65,7 @@ public class TransactorDocTest {
|
||||||
@Test
|
@Test
|
||||||
public void counterTransactor() throws Exception {
|
public void counterTransactor() throws Exception {
|
||||||
ActorSystem system = ActorSystem.create("CounterTransactor");
|
ActorSystem system = ActorSystem.create("CounterTransactor");
|
||||||
ActorRef counter = system.actorOf(new Props(Counter.class));
|
ActorRef counter = system.actorOf(Props.create(Counter.class));
|
||||||
|
|
||||||
Timeout timeout = new Timeout(5, SECONDS);
|
Timeout timeout = new Timeout(5, SECONDS);
|
||||||
Coordinated coordinated = new Coordinated(timeout);
|
Coordinated coordinated = new Coordinated(timeout);
|
||||||
|
|
@ -81,8 +81,8 @@ public class TransactorDocTest {
|
||||||
@Test
|
@Test
|
||||||
public void friendlyCounterTransactor() throws Exception {
|
public void friendlyCounterTransactor() throws Exception {
|
||||||
ActorSystem system = ActorSystem.create("FriendlyCounterTransactor");
|
ActorSystem system = ActorSystem.create("FriendlyCounterTransactor");
|
||||||
ActorRef friend = system.actorOf(new Props(Counter.class));
|
ActorRef friend = system.actorOf(Props.create(Counter.class));
|
||||||
ActorRef friendlyCounter = system.actorOf(new Props(FriendlyCounter.class));
|
ActorRef friendlyCounter = system.actorOf(Props.create(FriendlyCounter.class));
|
||||||
|
|
||||||
Timeout timeout = new Timeout(5, SECONDS);
|
Timeout timeout = new Timeout(5, SECONDS);
|
||||||
Coordinated coordinated = new Coordinated(timeout);
|
Coordinated coordinated = new Coordinated(timeout);
|
||||||
|
|
|
||||||
|
|
@ -68,6 +68,7 @@ public class ZeromqDocTestBase {
|
||||||
system.shutdown();
|
system.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unused")
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateCreateSocket() {
|
public void demonstrateCreateSocket() {
|
||||||
Assume.assumeTrue(checkZeroMQInstallation());
|
Assume.assumeTrue(checkZeroMQInstallation());
|
||||||
|
|
@ -78,7 +79,7 @@ public class ZeromqDocTestBase {
|
||||||
//#pub-socket
|
//#pub-socket
|
||||||
|
|
||||||
//#sub-socket
|
//#sub-socket
|
||||||
ActorRef listener = system.actorOf(new Props(ListenerActor.class));
|
ActorRef listener = system.actorOf(Props.create(ListenerActor.class));
|
||||||
ActorRef subSocket = ZeroMQExtension.get(system).newSubSocket(
|
ActorRef subSocket = ZeroMQExtension.get(system).newSubSocket(
|
||||||
new Connect("tcp://127.0.0.1:1233"),
|
new Connect("tcp://127.0.0.1:1233"),
|
||||||
new Listener(listener), Subscribe.all());
|
new Listener(listener), Subscribe.all());
|
||||||
|
|
@ -115,17 +116,17 @@ public class ZeromqDocTestBase {
|
||||||
|
|
||||||
//#health2
|
//#health2
|
||||||
|
|
||||||
system.actorOf(new Props(HealthProbe.class), "health");
|
system.actorOf(Props.create(HealthProbe.class), "health");
|
||||||
//#health2
|
//#health2
|
||||||
|
|
||||||
//#logger2
|
//#logger2
|
||||||
|
|
||||||
system.actorOf(new Props(Logger.class), "logger");
|
system.actorOf(Props.create(Logger.class), "logger");
|
||||||
//#logger2
|
//#logger2
|
||||||
|
|
||||||
//#alerter2
|
//#alerter2
|
||||||
|
|
||||||
system.actorOf(new Props(HeapAlerter.class), "alerter");
|
system.actorOf(Props.create(HeapAlerter.class), "alerter");
|
||||||
//#alerter2
|
//#alerter2
|
||||||
|
|
||||||
// Let it run for a while to see some output.
|
// Let it run for a while to see some output.
|
||||||
|
|
@ -159,6 +160,7 @@ public class ZeromqDocTestBase {
|
||||||
static
|
static
|
||||||
//#health
|
//#health
|
||||||
public class Heap implements Serializable {
|
public class Heap implements Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public final long timestamp;
|
public final long timestamp;
|
||||||
public final long used;
|
public final long used;
|
||||||
public final long max;
|
public final long max;
|
||||||
|
|
@ -174,6 +176,7 @@ public class ZeromqDocTestBase {
|
||||||
static
|
static
|
||||||
//#health
|
//#health
|
||||||
public class Load implements Serializable {
|
public class Load implements Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
public final long timestamp;
|
public final long timestamp;
|
||||||
public final double loadAverage;
|
public final double loadAverage;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -32,7 +32,7 @@ Creating Actors
|
||||||
Defining an Actor class
|
Defining an Actor class
|
||||||
-----------------------
|
-----------------------
|
||||||
|
|
||||||
Actor in Java are implemented by extending the ``UntypedActor`` class and implementing the
|
Actors in Java are implemented by extending the ``UntypedActor`` class and implementing the
|
||||||
:meth:`onReceive` method. This method takes the message as a parameter.
|
:meth:`onReceive` method. This method takes the message as a parameter.
|
||||||
|
|
||||||
Here is an example:
|
Here is an example:
|
||||||
|
|
@ -42,84 +42,132 @@ Here is an example:
|
||||||
Props
|
Props
|
||||||
-----
|
-----
|
||||||
|
|
||||||
``Props`` is a configuration class to specify options for the creation
|
:class:`Props` is a configuration class to specify options for the creation
|
||||||
of actors. Here are some examples on how to create a ``Props`` instance.
|
of actors, think of it as an immutable and thus freely shareable recipe for
|
||||||
|
creating an actor including associated deployment information (e.g. which
|
||||||
|
dispatcher to use, see more below). Here are some examples of how to create a
|
||||||
|
:class:`Props` instance.
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-props
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-props-config
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-props-config
|
||||||
|
|
||||||
|
The last line shows how to pass constructor arguments to the :class:`Actor`
|
||||||
|
being created. The presence of a matching constructor is 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):
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-untypedActor
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.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 instance’s ``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
|
||||||
|
actor’s ``this`` reference into :class:`Props`!
|
||||||
|
|
||||||
|
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 actor’s constructor signature at a later point,
|
||||||
|
where compiler checks will allow this modification to be done with greater
|
||||||
|
confidence than without.
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#props-factory
|
||||||
|
|
||||||
Creating Actors with Props
|
Creating Actors with Props
|
||||||
--------------------------
|
--------------------------
|
||||||
|
|
||||||
Actors are created by passing in a ``Props`` instance into the ``actorOf`` factory method.
|
Actors are created by passing a :class:`Props` instance into the
|
||||||
|
:meth:`actorOf` factory method which is available on :class:`ActorSystem` and
|
||||||
|
:class:`ActorContext`.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-props
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-actorRef
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#system-actorOf
|
||||||
|
|
||||||
|
Using the :class:`ActorSystem` will create top-level actors, supervised by the
|
||||||
|
actor system’s provided guardian actor, while using an actor’s context will
|
||||||
|
create a child actor.
|
||||||
|
|
||||||
Creating Actors with default constructor
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#context-actorOf
|
||||||
----------------------------------------
|
:exclude: plus-some-behavior
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
It is recommended to create a hierarchy of children, grand-children and so on
|
||||||
:include: imports
|
such that it fits the logical failure-handling structure of the application,
|
||||||
|
see :ref:`actor-systems`.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
The call to :meth:`actorOf` returns an instance of :class:`ActorRef`. This is a
|
||||||
:include: system-actorOf
|
handle to the actor instance and the only way to interact with it. The
|
||||||
|
:class:`ActorRef` is immutable and has a one to one relationship with the Actor
|
||||||
|
it represents. The :class:`ActorRef` is also serializable and network-aware.
|
||||||
|
This means that you can serialize it, send it over the wire and use it on a
|
||||||
|
remote host and it will still be representing the same Actor on the original
|
||||||
|
node, across the network.
|
||||||
|
|
||||||
The call to :meth:`actorOf` returns an instance of ``ActorRef``. This is a handle to
|
The name parameter is optional, but you should preferably name your actors,
|
||||||
the ``UntypedActor`` instance which you can use to interact with the ``UntypedActor``. The
|
since that is used in log messages and for identifying actors. The name must
|
||||||
``ActorRef`` is immutable and has a one to one relationship with the Actor it
|
not be empty or start with ``$``, but it may contain URL encoded characters
|
||||||
represents. The ``ActorRef`` is also serializable and network-aware. This means
|
(eg. ``%20`` for a blank space). If the given name is already in use by
|
||||||
that you can serialize it, send it over the wire and use it on a remote host and
|
another child to the same parent an `InvalidActorNameException` is thrown.
|
||||||
it will still be representing the same Actor on the original node, across the
|
|
||||||
network.
|
|
||||||
|
|
||||||
In the above example the actor was created from the system. It is also possible
|
|
||||||
to create actors from other actors with the actor ``context``. The difference is
|
|
||||||
how the supervisor hierarchy is arranged. When using the context the current actor
|
|
||||||
will be supervisor of the created child actor. When using the system it will be
|
|
||||||
a top level actor, that is supervised by the system (internal guardian actor).
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/FirstUntypedActor.java#context-actorOf
|
|
||||||
|
|
||||||
The name parameter is optional, but you should preferably name your actors, since
|
|
||||||
that is used in log messages and for identifying actors. The name must not be empty
|
|
||||||
or start with ``$``, but it may contain URL encoded characters (eg. ``%20`` for a blank space).
|
|
||||||
If the given name is already in use by another child to the
|
|
||||||
same parent actor an `InvalidActorNameException` is thrown.
|
|
||||||
|
|
||||||
Actors are automatically started asynchronously when created.
|
Actors are automatically started asynchronously when created.
|
||||||
When you create the ``UntypedActor`` then it will automatically call the ``preStart``
|
|
||||||
callback method on the ``UntypedActor`` class. This is an excellent place to
|
|
||||||
add initialization code for the actor.
|
|
||||||
|
|
||||||
.. code-block:: java
|
.. _actor-create-factory:
|
||||||
|
|
||||||
@Override
|
Creating Actors with Factory Methods
|
||||||
public void preStart() {
|
------------------------------------
|
||||||
... // initialization code
|
|
||||||
}
|
|
||||||
|
|
||||||
Creating Actors with non-default constructor
|
If your UntypedActor has a constructor that takes parameters then those need to
|
||||||
--------------------------------------------
|
be part of the :class:`Props` as well, as described `above <Props>`_. But there
|
||||||
|
are cases when a factory method must be used, for example when the actual
|
||||||
|
constructor arguments are determined by a dependency injection framework.
|
||||||
|
|
||||||
If your UntypedActor has a constructor that takes parameters then you can't create it using
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-indirect
|
||||||
'actorOf(new Props(clazz))'. Then you can instead pass in 'new Props(new UntypedActorFactory() {..})'
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
||||||
in which you can create the Actor in any way you like.
|
:include: creating-indirectly
|
||||||
|
:exclude: obtain-fresh-Actor-instance-from-DI-framework
|
||||||
Here is an example:
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-constructor
|
|
||||||
|
|
||||||
This way of creating the Actor is also great for integrating with Dependency Injection
|
|
||||||
(DI) frameworks like Guice or Spring.
|
|
||||||
|
|
||||||
.. warning::
|
.. warning::
|
||||||
|
|
||||||
You might be tempted at times to offer an ``UntypedActor`` factory which
|
You might be tempted at times to offer an :class:`IndirectActorProducer`
|
||||||
always returns the same instance, e.g. by using a static field. This is not
|
which always returns the same instance, e.g. by using a static field. This is
|
||||||
supported, as it goes against the meaning of an actor restart, which is
|
not supported, as it goes against the meaning of an actor restart, which is
|
||||||
described here: :ref:`supervision-restart`.
|
described here: :ref:`supervision-restart`.
|
||||||
|
|
||||||
|
When using a dependency injection framework, actor beans *MUST NOT* have
|
||||||
|
singleton scope.
|
||||||
|
|
||||||
UntypedActor API
|
UntypedActor API
|
||||||
================
|
================
|
||||||
|
|
||||||
|
|
@ -180,6 +228,7 @@ termination (see `Stopping Actors`_). This service is provided by the
|
||||||
Registering a monitor is easy (see fourth line, the rest is for demonstrating
|
Registering a monitor is easy (see fourth line, the rest is for demonstrating
|
||||||
the whole functionality):
|
the whole functionality):
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-terminated
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#watch
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#watch
|
||||||
|
|
||||||
It should be noted that the :class:`Terminated` message is generated
|
It should be noted that the :class:`Terminated` message is generated
|
||||||
|
|
@ -206,14 +255,15 @@ Start Hook
|
||||||
|
|
||||||
Right after starting the actor, its :meth:`preStart` method is invoked.
|
Right after starting the actor, its :meth:`preStart` method is invoked.
|
||||||
|
|
||||||
::
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#preStart
|
||||||
|
|
||||||
@Override
|
|
||||||
public void preStart() {
|
|
||||||
// registering with other actors
|
|
||||||
someService.tell(Register(getSelf());
|
|
||||||
}
|
|
||||||
|
|
||||||
|
This method is called when the actor is first created. During restarts it is
|
||||||
|
called by the default implementation of :meth:`postRestart`, which means that
|
||||||
|
by overriding that method you can choose whether the initialization code in
|
||||||
|
this method is called only exactly once for this actor or for every restart.
|
||||||
|
Initialization code which is part of the actor’s constructor will always be
|
||||||
|
called when an instance of the actor class is created, which happens at every
|
||||||
|
restart.
|
||||||
|
|
||||||
Restart Hooks
|
Restart Hooks
|
||||||
-------------
|
-------------
|
||||||
|
|
@ -280,12 +330,9 @@ are used by the system to look up actors, e.g. when a remote message is
|
||||||
received and the recipient is searched, but they are also useful more directly:
|
received and the recipient is searched, but they are also useful more directly:
|
||||||
actors may look up other actors by specifying absolute or relative
|
actors may look up other actors by specifying absolute or relative
|
||||||
paths—logical or physical—and receive back an :class:`ActorSelection` with the
|
paths—logical or physical—and receive back an :class:`ActorSelection` with the
|
||||||
result::
|
result:
|
||||||
|
|
||||||
// will look up this absolute path
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#selection-local
|
||||||
getContext().actorSelection("/user/serviceA/actor");
|
|
||||||
// will look up sibling beneath same supervisor
|
|
||||||
getContext().actorSelection("../joe");
|
|
||||||
|
|
||||||
The supplied path is parsed as a :class:`java.net.URI`, which basically means
|
The supplied path is parsed as a :class:`java.net.URI`, which basically means
|
||||||
that it is split on ``/`` into path elements. If the path starts with ``/``, it
|
that it is split on ``/`` into path elements. If the path starts with ``/``, it
|
||||||
|
|
@ -297,12 +344,9 @@ It should be noted that the ``..`` in actor paths here always means the logical
|
||||||
structure, i.e. the supervisor.
|
structure, i.e. the supervisor.
|
||||||
|
|
||||||
The path elements of an actor selection may contain wildcard patterns allowing for
|
The path elements of an actor selection may contain wildcard patterns allowing for
|
||||||
broadcasting of messages to that section::
|
broadcasting of messages to that section:
|
||||||
|
|
||||||
// will look all children to serviceB with names starting with worker
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#selection-wildcard
|
||||||
getContext().actorSelection("/user/serviceB/worker*");
|
|
||||||
// will look up all siblings beneath same supervisor
|
|
||||||
getContext().actorSelection("../*");
|
|
||||||
|
|
||||||
Messages can be sent via the :class:`ActorSelection` and the path of the
|
Messages can be sent via the :class:`ActorSelection` and the path of the
|
||||||
:class:`ActorSelection` is looked up when delivering each message. If the selection
|
:class:`ActorSelection` is looked up when delivering each message. If the selection
|
||||||
|
|
@ -315,11 +359,11 @@ and automatically reply to with a ``ActorIdentity`` message containing the
|
||||||
:class:`ActorRef`.
|
:class:`ActorRef`.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
||||||
:include: identify-imports,identify
|
:include: import-identify,identify
|
||||||
|
|
||||||
Remote actor addresses may also be looked up, if :ref:`remoting <remoting-java>` is enabled::
|
Remote actor addresses may also be looked up, if :ref:`remoting <remoting-java>` is enabled:
|
||||||
|
|
||||||
getContext().actorSelection("akka.tcp://app@otherhost:1234/user/serviceB");
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#selection-remote
|
||||||
|
|
||||||
An example demonstrating remote actor look-up is given in :ref:`remote-lookup-sample-java`.
|
An example demonstrating remote actor look-up is given in :ref:`remote-lookup-sample-java`.
|
||||||
|
|
||||||
|
|
@ -369,28 +413,24 @@ In all these methods you have the option of passing along your own ``ActorRef``.
|
||||||
Make it a practice of doing so because it will allow the receiver actors to be able to respond
|
Make it a practice of doing so because it will allow the receiver actors to be able to respond
|
||||||
to your message, since the sender reference is sent along with the message.
|
to your message, since the sender reference is sent along with the message.
|
||||||
|
|
||||||
|
.. _actors-tell-sender-java:
|
||||||
|
|
||||||
Tell: Fire-forget
|
Tell: Fire-forget
|
||||||
-----------------
|
-----------------
|
||||||
|
|
||||||
This is the preferred way of sending messages. No blocking waiting for a
|
This is the preferred way of sending messages. No blocking waiting for a
|
||||||
message. This gives the best concurrency and scalability characteristics.
|
message. This gives the best concurrency and scalability characteristics.
|
||||||
|
|
||||||
.. code-block:: java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#tell
|
||||||
|
|
||||||
actor.tell("Hello");
|
The sender reference is passed along with the message and available within the
|
||||||
|
receiving actor via its :meth:`getSender()` method while processing this
|
||||||
.. _actors-tell-sender-java:
|
message. Inside of an actor it is usually :meth:`getSelf` who shall be the
|
||||||
|
sender, but there can be cases where replies shall be routed to some other
|
||||||
Or with the sender reference passed along with the message and available to the receiving Actor
|
actor—e.g. the parent—in which the second argument to :meth:`tell` would be a
|
||||||
in its ``getSender: ActorRef`` member field. The target actor can use this
|
different one. Outside of an actor and if no reply is needed the second
|
||||||
to reply to the original sender, by using ``getSender().tell(replyMsg)``.
|
argument can be ``null``; if a reply is needed outside of an actor you can use
|
||||||
|
the ask-pattern described next..
|
||||||
.. code-block:: java
|
|
||||||
|
|
||||||
actor.tell("Hello", getSelf());
|
|
||||||
|
|
||||||
If invoked without the sender parameter the sender will be
|
|
||||||
:obj:`deadLetters` actor reference in the target actor.
|
|
||||||
|
|
||||||
Ask: Send-And-Receive-Future
|
Ask: Send-And-Receive-Future
|
||||||
----------------------------
|
----------------------------
|
||||||
|
|
@ -398,8 +438,7 @@ Ask: Send-And-Receive-Future
|
||||||
The ``ask`` pattern involves actors as well as futures, hence it is offered as
|
The ``ask`` pattern involves actors as well as futures, hence it is offered as
|
||||||
a use pattern rather than a method on :class:`ActorRef`:
|
a use pattern rather than a method on :class:`ActorRef`:
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-askPipe
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-ask
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#ask-pipe
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#ask-pipe
|
||||||
|
|
||||||
This example demonstrates ``ask`` together with the ``pipe`` pattern on
|
This example demonstrates ``ask`` together with the ``pipe`` pattern on
|
||||||
|
|
@ -454,9 +493,7 @@ through a 'mediator'. This can be useful when writing actors that work as
|
||||||
routers, load-balancers, replicators etc.
|
routers, load-balancers, replicators etc.
|
||||||
You need to pass along your context variable as well.
|
You need to pass along your context variable as well.
|
||||||
|
|
||||||
.. code-block:: java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#forward
|
||||||
|
|
||||||
myActor.forward(message, getContext());
|
|
||||||
|
|
||||||
Receive messages
|
Receive messages
|
||||||
================
|
================
|
||||||
|
|
@ -484,12 +521,8 @@ for replying later, or passing on to other actors. If there is no sender (a
|
||||||
message was sent without an actor or future context) then the sender
|
message was sent without an actor or future context) then the sender
|
||||||
defaults to a 'dead-letter' actor ref.
|
defaults to a 'dead-letter' actor ref.
|
||||||
|
|
||||||
.. code-block:: java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#reply
|
||||||
|
:exclude: calculate-result
|
||||||
public void onReceive(Object request) {
|
|
||||||
String result = process(request);
|
|
||||||
getSender().tell(result); // will have dead-letter actor as default
|
|
||||||
}
|
|
||||||
|
|
||||||
Receive timeout
|
Receive timeout
|
||||||
===============
|
===============
|
||||||
|
|
@ -506,7 +539,7 @@ timeout there must have been an idle period beforehand as configured via this me
|
||||||
Once set, the receive timeout stays in effect (i.e. continues firing repeatedly after inactivity
|
Once set, the receive timeout stays in effect (i.e. continues firing repeatedly after inactivity
|
||||||
periods). Pass in `Duration.Undefined` to switch off this feature.
|
periods). Pass in `Duration.Undefined` to switch off this feature.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/MyReceivedTimeoutUntypedActor.java#receive-timeout
|
.. includecode:: code/docs/actor/MyReceiveTimeoutUntypedActor.java#receive-timeout
|
||||||
|
|
||||||
.. _stopping-actors-java:
|
.. _stopping-actors-java:
|
||||||
|
|
||||||
|
|
@ -543,12 +576,8 @@ whole system.
|
||||||
The :meth:`postStop()` hook is invoked after an actor is fully stopped. This
|
The :meth:`postStop()` hook is invoked after an actor is fully stopped. This
|
||||||
enables cleaning up of resources:
|
enables cleaning up of resources:
|
||||||
|
|
||||||
.. code-block:: java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#postStop
|
||||||
|
:exclude: clean-up-resources-here
|
||||||
@Override
|
|
||||||
public void postStop() {
|
|
||||||
// close some file or database connection
|
|
||||||
}
|
|
||||||
|
|
||||||
.. note::
|
.. note::
|
||||||
|
|
||||||
|
|
@ -570,9 +599,6 @@ in the mailbox.
|
||||||
|
|
||||||
Use it like this:
|
Use it like this:
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
|
||||||
:include: import-actors
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
||||||
:include: poison-pill
|
:include: poison-pill
|
||||||
|
|
||||||
|
|
@ -659,6 +685,7 @@ order as they have been received originally.
|
||||||
|
|
||||||
Here is an example of the ``UntypedActorWithStash`` class in action:
|
Here is an example of the ``UntypedActorWithStash`` class in action:
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-stash
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#stash
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#stash
|
||||||
|
|
||||||
Invoking ``stash()`` adds the current message (the message that the
|
Invoking ``stash()`` adds the current message (the message that the
|
||||||
|
|
@ -702,9 +729,6 @@ See :ref:`supervision-directives` for more information.
|
||||||
|
|
||||||
Use ``Kill`` like this:
|
Use ``Kill`` like this:
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
|
||||||
:include: import-actors
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
||||||
:include: kill
|
:include: kill
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,26 @@ simple, mechanical source-level changes in client code.
|
||||||
When migrating from 1.3.x to 2.1.x you should first follow the instructions for
|
When migrating from 1.3.x to 2.1.x you should first follow the instructions for
|
||||||
migrating :ref:`1.3.x to 2.0.x <migration-2.0>` and then :ref:`2.0.x to 2.1.x <migration-2.1>`.
|
migrating :ref:`1.3.x to 2.0.x <migration-2.0>` and then :ref:`2.0.x to 2.1.x <migration-2.1>`.
|
||||||
|
|
||||||
|
Deprecated Closure-Taking Props
|
||||||
|
===============================
|
||||||
|
|
||||||
|
:class:`Props` instances used to contain a closure which produces an
|
||||||
|
:class:`Actor` instance when invoked. This approach is flawed in that closures
|
||||||
|
are usually created in-line and thus carry a reference to their enclosing
|
||||||
|
object; this is not well known among programmers, in particular it can be
|
||||||
|
surprising that innocent-looking actor creation should not be serializable if
|
||||||
|
the e.g. the enclosing class is an actor.
|
||||||
|
|
||||||
|
Thus we have decided to deprecate ``Props(new MyActor(...))`` and
|
||||||
|
:class:`UntypedActorFactory` in favor of basing :class:`Props` on a
|
||||||
|
:class:`Class` and a sequence of constructor arguments. This has the added
|
||||||
|
benefit of allowing easier integration with dependency injection frameworks,
|
||||||
|
see :ref:`actor-create-factory`.
|
||||||
|
|
||||||
|
The deprecated methods will be retained until the possibility of reintroducing
|
||||||
|
a similar syntax in a safe fashion has been properly researched (in case of
|
||||||
|
Scala it might be possible to use macros to this effect).
|
||||||
|
|
||||||
Immutable everywhere
|
Immutable everywhere
|
||||||
====================
|
====================
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -43,114 +43,145 @@ Here is an example:
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala
|
.. includecode:: code/docs/actor/ActorDocSpec.scala
|
||||||
:include: imports1,my-actor
|
:include: imports1,my-actor
|
||||||
|
|
||||||
Please note that the Akka Actor ``receive`` message loop is exhaustive, which is
|
Please note that the Akka Actor ``receive`` message loop is exhaustive, which
|
||||||
different compared to Erlang and Scala Actors. This means that you need to
|
is different compared to Erlang and the late Scala Actors. This means that you
|
||||||
provide a pattern match for all messages that it can accept and if you want to
|
need to provide a pattern match for all messages that it can accept and if you
|
||||||
be able to handle unknown messages then you need to have a default case as in
|
want to be able to handle unknown messages then you need to have a default case
|
||||||
the example above. Otherwise an ``akka.actor.UnhandledMessage(message, sender, recipient)`` will be
|
as in the example above. Otherwise an ``akka.actor.UnhandledMessage(message,
|
||||||
published to the ``ActorSystem``'s ``EventStream``.
|
sender, recipient)`` will be published to the ``ActorSystem``'s
|
||||||
|
``EventStream``.
|
||||||
|
|
||||||
|
Note further that the return type of the behavior defined above is ``Unit``; if
|
||||||
|
the actor shall reply to the received message then this must be done explicitly
|
||||||
|
as explained below.
|
||||||
|
|
||||||
The result of the :meth:`receive` method is a partial function object, which is
|
The result of the :meth:`receive` method is a partial function object, which is
|
||||||
stored within the actor as its “initial behavior”, see `Become/Unbecome`_ for
|
stored within the actor as its “initial behavior”, see `Become/Unbecome`_ for
|
||||||
further information on changing the behavior of an actor after its
|
further information on changing the behavior of an actor after its
|
||||||
construction.
|
construction.
|
||||||
|
|
||||||
Creating Actors with default constructor
|
|
||||||
----------------------------------------
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala
|
|
||||||
:include: imports2,system-actorOf
|
|
||||||
|
|
||||||
The call to :meth:`actorOf` returns an instance of ``ActorRef``. This is a handle to
|
|
||||||
the ``Actor`` instance which you can use to interact with the ``Actor``. The
|
|
||||||
``ActorRef`` is immutable and has a one to one relationship with the Actor it
|
|
||||||
represents. The ``ActorRef`` is also serializable and network-aware. This means
|
|
||||||
that you can serialize it, send it over the wire and use it on a remote host and
|
|
||||||
it will still be representing the same Actor on the original node, across the
|
|
||||||
network.
|
|
||||||
|
|
||||||
In the above example the actor was created from the system. It is also possible
|
|
||||||
to create actors from other actors with the actor ``context``. The difference is
|
|
||||||
how the supervisor hierarchy is arranged. When using the context the current actor
|
|
||||||
will be supervisor of the created child actor. When using the system it will be
|
|
||||||
a top level actor, that is supervised by the system (internal guardian actor).
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#context-actorOf
|
|
||||||
|
|
||||||
The name parameter is optional, but you should preferably name your actors, since
|
|
||||||
that is used in log messages and for identifying actors. The name must not be empty
|
|
||||||
or start with ``$``, but it may contain URL encoded characters (eg. ``%20`` for a blank space).
|
|
||||||
If the given name is already in use by another child to the
|
|
||||||
same parent actor an `InvalidActorNameException` is thrown.
|
|
||||||
|
|
||||||
Actors are automatically started asynchronously when created.
|
|
||||||
When you create the ``Actor`` then it will automatically call the ``preStart``
|
|
||||||
callback method on the ``Actor`` trait. This is an excellent place to
|
|
||||||
add initialization code for the actor.
|
|
||||||
|
|
||||||
.. code-block:: scala
|
|
||||||
|
|
||||||
override def preStart() = {
|
|
||||||
... // initialization code
|
|
||||||
}
|
|
||||||
|
|
||||||
Creating Actors with non-default constructor
|
|
||||||
--------------------------------------------
|
|
||||||
|
|
||||||
If your Actor has a constructor that takes parameters then you can't create it
|
|
||||||
using ``actorOf(Props[TYPE])``. Instead you can use a variant of ``actorOf`` that takes
|
|
||||||
a call-by-name block in which you can create the Actor in any way you like.
|
|
||||||
|
|
||||||
Here is an example:
|
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-constructor
|
|
||||||
|
|
||||||
.. warning::
|
|
||||||
|
|
||||||
You might be tempted at times to offer an ``Actor`` factory which always
|
|
||||||
returns the same instance, e.g. by using a ``lazy val`` or an
|
|
||||||
``object ... extends Actor``. This is not supported, as it goes against the
|
|
||||||
meaning of an actor restart, which is described here:
|
|
||||||
:ref:`supervision-restart`.
|
|
||||||
|
|
||||||
.. warning::
|
|
||||||
|
|
||||||
Also avoid passing mutable state into the constructor of the Actor, since
|
|
||||||
the call-by-name block can be executed by another thread.
|
|
||||||
|
|
||||||
Props
|
Props
|
||||||
-----
|
-----
|
||||||
|
|
||||||
``Props`` is a configuration class to specify options for the creation
|
:class:`Props` is a configuration class to specify options for the creation
|
||||||
of actors. Here are some examples on how to create a ``Props`` instance.
|
of actors, think of it as an immutable and thus freely shareable recipe for
|
||||||
|
creating an actor including associated deployment information (e.g. which
|
||||||
|
dispatcher to use, see more below). Here are some examples of how to create a
|
||||||
|
:class:`Props` instance.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props-config
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props
|
||||||
|
|
||||||
|
The last line shows how to pass constructor arguments to the :class:`Actor`
|
||||||
|
being created. The presence of a matching constructor is 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):
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props-deprecated
|
||||||
|
|
||||||
|
The last one is deprecated because its functionality is available in full
|
||||||
|
through :meth:`Props.apply()`.
|
||||||
|
|
||||||
|
The first three are deprecated because the captured closure is 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.apply(clazz, args)` 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 declared within a top-level ``object``
|
||||||
|
then the enclosing instance’s ``this`` reference needs to be passed as the
|
||||||
|
first argument).
|
||||||
|
|
||||||
|
.. warning::
|
||||||
|
|
||||||
|
Declaring one actor within another is very dangerous and breaks actor
|
||||||
|
encapsulation. Never pass an actor’s ``this`` reference into :class:`Props`!
|
||||||
|
|
||||||
|
Recommended Practices
|
||||||
|
^^^^^^^^^^^^^^^^^^^^^
|
||||||
|
|
||||||
|
It is a good idea to provide factory methods on the companion object of each
|
||||||
|
:class:`Actor` 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 actor’s constructor signature at a later point, where
|
||||||
|
compiler checks will allow this modification to be done with greater confidence
|
||||||
|
than without.
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#props-factory
|
||||||
|
|
||||||
Creating Actors with Props
|
Creating Actors with Props
|
||||||
--------------------------
|
--------------------------
|
||||||
|
|
||||||
Actors are created by passing in a ``Props`` instance into the ``actorOf`` factory method.
|
Actors are created by passing a :class:`Props` instance into the
|
||||||
|
:meth:`actorOf` factory method which is available on :class:`ActorSystem` and
|
||||||
|
:class:`ActorContext`.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#system-actorOf
|
||||||
|
|
||||||
|
Using the :class:`ActorSystem` will create top-level actors, supervised by the
|
||||||
|
actor system’s provided guardian actor, while using an actor’s context will
|
||||||
|
create a child actor.
|
||||||
|
|
||||||
Creating Actors using anonymous classes
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#context-actorOf
|
||||||
---------------------------------------
|
:exclude: plus-some-behavior
|
||||||
|
|
||||||
When spawning actors for specific sub-tasks from within an actor, it may be convenient to include the code to be executed directly in place, using an anonymous class.
|
It is recommended to create a hierarchy of children, grand-children and so on
|
||||||
|
such that it fits the logical failure-handling structure of the application,
|
||||||
|
see :ref:`actor-systems`.
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#anonymous-actor
|
The call to :meth:`actorOf` returns an instance of :class:`ActorRef`. This is a
|
||||||
|
handle to the actor instance and the only way to interact with it. The
|
||||||
|
:class:`ActorRef` is immutable and has a one to one relationship with the Actor
|
||||||
|
it represents. The :class:`ActorRef` is also serializable and network-aware.
|
||||||
|
This means that you can serialize it, send it over the wire and use it on a
|
||||||
|
remote host and it will still be representing the same Actor on the original
|
||||||
|
node, across the network.
|
||||||
|
|
||||||
|
The name parameter is optional, but you should preferably name your actors,
|
||||||
|
since that is used in log messages and for identifying actors. The name must
|
||||||
|
not be empty or start with ``$``, but it may contain URL encoded characters
|
||||||
|
(eg. ``%20`` for a blank space). If the given name is already in use by
|
||||||
|
another child to the same parent an `InvalidActorNameException` is thrown.
|
||||||
|
|
||||||
|
Actors are automatically started asynchronously when created.
|
||||||
|
|
||||||
|
Creating Actors with Factory Methods
|
||||||
|
------------------------------------
|
||||||
|
|
||||||
|
If your UntypedActor has a constructor that takes parameters then those need to
|
||||||
|
be part of the :class:`Props` as well, as described `above <Props>`_. But there
|
||||||
|
are cases when a factory method must be used, for example when the actual
|
||||||
|
constructor arguments are determined by a dependency injection framework.
|
||||||
|
|
||||||
|
.. includecode:: code/docs/actor/ActorDocSpec.scala
|
||||||
|
:include: creating-indirectly
|
||||||
|
:exclude: obtain-fresh-Actor-instance-from-DI-framework
|
||||||
|
|
||||||
.. warning::
|
.. warning::
|
||||||
|
|
||||||
In this case you need to carefully avoid closing over the containing actor’s
|
You might be tempted at times to offer an :class:`IndirectActorProducer`
|
||||||
reference, i.e. do not call methods on the enclosing actor from within the
|
which always returns the same instance, e.g. by using a ``lazy val``. This is
|
||||||
anonymous Actor class. This would break the actor encapsulation and may
|
not supported, as it goes against the meaning of an actor restart, which is
|
||||||
introduce synchronization bugs and race conditions because the other actor’s
|
described here: :ref:`supervision-restart`.
|
||||||
code will be scheduled concurrently to the enclosing actor. Unfortunately
|
|
||||||
there is not yet a way to detect these illegal accesses at compile time.
|
When using a dependency injection framework, actor beans *MUST NOT* have
|
||||||
See also: :ref:`jmm-shared-state`
|
singleton scope.
|
||||||
|
|
||||||
The Actor DSL
|
The Actor DSL
|
||||||
-------------
|
-------------
|
||||||
|
|
@ -268,15 +299,9 @@ You can import the members in the :obj:`context` to avoid prefixing access with
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#import-context
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#import-context
|
||||||
|
|
||||||
The remaining visible methods are user-overridable life-cycle hooks which are
|
The remaining visible methods are user-overridable life-cycle hooks which are
|
||||||
described in the following::
|
described in the following:
|
||||||
|
|
||||||
def preStart() {}
|
.. includecode:: ../../../akka-actor/src/main/scala/akka/actor/Actor.scala#lifecycle-hooks
|
||||||
def preRestart(reason: Throwable, message: Option[Any]) {
|
|
||||||
context.children foreach (context.stop(_))
|
|
||||||
postStop()
|
|
||||||
}
|
|
||||||
def postRestart(reason: Throwable) { preStart() }
|
|
||||||
def postStop() {}
|
|
||||||
|
|
||||||
The implementations shown above are the defaults provided by the :class:`Actor`
|
The implementations shown above are the defaults provided by the :class:`Actor`
|
||||||
trait.
|
trait.
|
||||||
|
|
@ -319,13 +344,15 @@ Start Hook
|
||||||
|
|
||||||
Right after starting the actor, its :meth:`preStart` method is invoked.
|
Right after starting the actor, its :meth:`preStart` method is invoked.
|
||||||
|
|
||||||
::
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#preStart
|
||||||
|
|
||||||
override def preStart() {
|
|
||||||
// registering with other actors
|
|
||||||
someService ! Register(self)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
This method is called when the actor is first created. During restarts it is
|
||||||
|
called by the default implementation of :meth:`postRestart`, which means that
|
||||||
|
by overriding that method you can choose whether the initialization code in
|
||||||
|
this method is called only exactly once for this actor or for every restart.
|
||||||
|
Initialization code which is part of the actor’s constructor will always be
|
||||||
|
called when an instance of the actor class is created, which happens at every
|
||||||
|
restart.
|
||||||
|
|
||||||
Restart Hooks
|
Restart Hooks
|
||||||
-------------
|
-------------
|
||||||
|
|
@ -390,12 +417,9 @@ are used by the system to look up actors, e.g. when a remote message is
|
||||||
received and the recipient is searched, but they are also useful more directly:
|
received and the recipient is searched, but they are also useful more directly:
|
||||||
actors may look up other actors by specifying absolute or relative
|
actors may look up other actors by specifying absolute or relative
|
||||||
paths—logical or physical—and receive back an :class:`ActorSelection` with the
|
paths—logical or physical—and receive back an :class:`ActorSelection` with the
|
||||||
result::
|
result:
|
||||||
|
|
||||||
// will look up this absolute path
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#selection-local
|
||||||
context.actorSelection("/user/serviceA/aggregator")
|
|
||||||
// will look up sibling beneath same supervisor
|
|
||||||
context.actorSelection("../joe")
|
|
||||||
|
|
||||||
The supplied path is parsed as a :class:`java.net.URI`, which basically means
|
The supplied path is parsed as a :class:`java.net.URI`, which basically means
|
||||||
that it is split on ``/`` into path elements. If the path starts with ``/``, it
|
that it is split on ``/`` into path elements. If the path starts with ``/``, it
|
||||||
|
|
@ -407,12 +431,9 @@ It should be noted that the ``..`` in actor paths here always means the logical
|
||||||
structure, i.e. the supervisor.
|
structure, i.e. the supervisor.
|
||||||
|
|
||||||
The path elements of an actor selection may contain wildcard patterns allowing for
|
The path elements of an actor selection may contain wildcard patterns allowing for
|
||||||
broadcasting of messages to that section::
|
broadcasting of messages to that section:
|
||||||
|
|
||||||
// will look all children to serviceB with names starting with worker
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#selection-wildcard
|
||||||
context.actorSelection("/user/serviceB/worker*")
|
|
||||||
// will look up all siblings beneath same supervisor
|
|
||||||
context.actorSelection("../*")
|
|
||||||
|
|
||||||
Messages can be sent via the :class:`ActorSelection` and the path of the
|
Messages can be sent via the :class:`ActorSelection` and the path of the
|
||||||
:class:`ActorSelection` is looked up when delivering each message. If the selection
|
:class:`ActorSelection` is looked up when delivering each message. If the selection
|
||||||
|
|
@ -426,9 +447,9 @@ and automatically reply to with a ``ActorIdentity`` message containing the
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#identify
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#identify
|
||||||
|
|
||||||
Remote actor addresses may also be looked up, if :ref:`remoting <remoting-scala>` is enabled::
|
Remote actor addresses may also be looked up, if :ref:`remoting <remoting-scala>` is enabled:
|
||||||
|
|
||||||
context.actorSelection("akka.tcp://app@otherhost:1234/user/serviceB")
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#selection-remote
|
||||||
|
|
||||||
An example demonstrating actor look-up is given in :ref:`remote-lookup-sample-scala`.
|
An example demonstrating actor look-up is given in :ref:`remote-lookup-sample-scala`.
|
||||||
|
|
||||||
|
|
@ -463,10 +484,6 @@ Here is an example:
|
||||||
// create a new case class message
|
// create a new case class message
|
||||||
val message = Register(user)
|
val message = Register(user)
|
||||||
|
|
||||||
Other good messages types are ``scala.Tuple2``, ``scala.List``, ``scala.Map``
|
|
||||||
which are all immutable and great for pattern matching.
|
|
||||||
|
|
||||||
|
|
||||||
Send messages
|
Send messages
|
||||||
=============
|
=============
|
||||||
|
|
||||||
|
|
@ -486,17 +503,15 @@ Message ordering is guaranteed on a per-sender basis.
|
||||||
a ``Promise`` into an ``ActorRef`` and it also needs to be reachable through
|
a ``Promise`` into an ``ActorRef`` and it also needs to be reachable through
|
||||||
remoting. So always prefer ``tell`` for performance, and only ``ask`` if you must.
|
remoting. So always prefer ``tell`` for performance, and only ``ask`` if you must.
|
||||||
|
|
||||||
|
.. _actors-tell-sender-scala:
|
||||||
|
|
||||||
Tell: Fire-forget
|
Tell: Fire-forget
|
||||||
-----------------
|
-----------------
|
||||||
|
|
||||||
This is the preferred way of sending messages. No blocking waiting for a
|
This is the preferred way of sending messages. No blocking waiting for a
|
||||||
message. This gives the best concurrency and scalability characteristics.
|
message. This gives the best concurrency and scalability characteristics.
|
||||||
|
|
||||||
.. code-block:: scala
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#tell
|
||||||
|
|
||||||
actor ! "hello"
|
|
||||||
|
|
||||||
.. _actors-tell-sender-scala:
|
|
||||||
|
|
||||||
If invoked from within an Actor, then the sending actor reference will be
|
If invoked from within an Actor, then the sending actor reference will be
|
||||||
implicitly passed along with the message and available to the receiving Actor
|
implicitly passed along with the message and available to the receiving Actor
|
||||||
|
|
@ -573,25 +588,16 @@ original sender address/reference is maintained even though the message is going
|
||||||
through a 'mediator'. This can be useful when writing actors that work as
|
through a 'mediator'. This can be useful when writing actors that work as
|
||||||
routers, load-balancers, replicators etc.
|
routers, load-balancers, replicators etc.
|
||||||
|
|
||||||
.. code-block:: scala
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#forward
|
||||||
|
|
||||||
myActor.forward(message)
|
|
||||||
|
|
||||||
|
|
||||||
Receive messages
|
Receive messages
|
||||||
================
|
================
|
||||||
|
|
||||||
An Actor has to implement the ``receive`` method to receive messages:
|
An Actor has to implement the ``receive`` method to receive messages:
|
||||||
|
|
||||||
.. code-block:: scala
|
.. includecode:: ../../../akka-actor/src/main/scala/akka/actor/Actor.scala#receive
|
||||||
|
|
||||||
def receive: PartialFunction[Any, Unit]
|
This method returns a ``PartialFunction``, e.g. a ‘match/case’ clause in
|
||||||
|
|
||||||
Note: Akka has an alias to the ``PartialFunction[Any, Unit]`` type called
|
|
||||||
``Receive`` (``akka.actor.Actor.Receive``), so you can use this type instead for
|
|
||||||
clarity. But most often you don't need to spell it out.
|
|
||||||
|
|
||||||
This method should return a ``PartialFunction``, e.g. a ‘match/case’ clause in
|
|
||||||
which the message can be matched against the different case clauses using Scala
|
which the message can be matched against the different case clauses using Scala
|
||||||
pattern matching. Here is an example:
|
pattern matching. Here is an example:
|
||||||
|
|
||||||
|
|
@ -669,11 +675,8 @@ whole system.
|
||||||
The :meth:`postStop()` hook is invoked after an actor is fully stopped. This
|
The :meth:`postStop()` hook is invoked after an actor is fully stopped. This
|
||||||
enables cleaning up of resources:
|
enables cleaning up of resources:
|
||||||
|
|
||||||
.. code-block:: scala
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#postStop
|
||||||
|
:exclude: clean-up-some-resources
|
||||||
override def postStop() = {
|
|
||||||
// close some file or database connection
|
|
||||||
}
|
|
||||||
|
|
||||||
.. note::
|
.. note::
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -37,11 +37,43 @@ case class Message(s: String)
|
||||||
|
|
||||||
//#context-actorOf
|
//#context-actorOf
|
||||||
class FirstActor extends Actor {
|
class FirstActor extends Actor {
|
||||||
val myActor = context.actorOf(Props[MyActor], name = "myactor")
|
val child = context.actorOf(Props[MyActor], name = "myChild")
|
||||||
//#context-actorOf
|
//#plus-some-behavior
|
||||||
def receive = {
|
def receive = {
|
||||||
case x ⇒ sender ! x
|
case x ⇒ sender ! x
|
||||||
}
|
}
|
||||||
|
//#plus-some-behavior
|
||||||
|
}
|
||||||
|
//#context-actorOf
|
||||||
|
|
||||||
|
class ActorWithArgs(arg: String) extends Actor {
|
||||||
|
def receive = { case _ ⇒ () }
|
||||||
|
}
|
||||||
|
|
||||||
|
class DemoActorWrapper extends Actor {
|
||||||
|
//#props-factory
|
||||||
|
object DemoActor {
|
||||||
|
/**
|
||||||
|
* Create Props for an actor of this type.
|
||||||
|
* @param name The name to be passed to this actor’s constructor.
|
||||||
|
* @return a Props for creating this actor, which can then be further configured
|
||||||
|
* (e.g. calling `.withDispatcher()` on it)
|
||||||
|
*/
|
||||||
|
def apply(name: String): Props = Props(classOf[DemoActor], name)
|
||||||
|
}
|
||||||
|
|
||||||
|
class DemoActor(name: String) extends Actor {
|
||||||
|
def receive = {
|
||||||
|
case x ⇒ // some behavior
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// ...
|
||||||
|
|
||||||
|
context.actorOf(DemoActor("hello"))
|
||||||
|
//#props-factory
|
||||||
|
|
||||||
|
def receive = Actor.emptyBehavior
|
||||||
}
|
}
|
||||||
|
|
||||||
class AnonymousActor extends Actor {
|
class AnonymousActor extends Actor {
|
||||||
|
|
@ -61,11 +93,21 @@ class AnonymousActor extends Actor {
|
||||||
//#anonymous-actor
|
//#anonymous-actor
|
||||||
}
|
}
|
||||||
|
|
||||||
//#system-actorOf
|
class Hook extends Actor {
|
||||||
object Main extends App {
|
var child: ActorRef = _
|
||||||
val system = ActorSystem("MySystem")
|
//#preStart
|
||||||
val myActor = system.actorOf(Props[MyActor], name = "myactor")
|
override def preStart() {
|
||||||
//#system-actorOf
|
child = context.actorOf(Props[MyActor], "child")
|
||||||
|
}
|
||||||
|
//#preStart
|
||||||
|
def receive = Actor.emptyBehavior
|
||||||
|
//#postStop
|
||||||
|
override def postStop() {
|
||||||
|
//#clean-up-some-resources
|
||||||
|
()
|
||||||
|
//#clean-up-some-resources
|
||||||
|
}
|
||||||
|
//#postStop
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReplyException extends Actor {
|
class ReplyException extends Actor {
|
||||||
|
|
@ -142,6 +184,7 @@ case class MyMsg(subject: String)
|
||||||
class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
|
|
||||||
"import context" in {
|
"import context" in {
|
||||||
|
new AnyRef {
|
||||||
//#import-context
|
//#import-context
|
||||||
class FirstActor extends Actor {
|
class FirstActor extends Actor {
|
||||||
import context._
|
import context._
|
||||||
|
|
@ -152,12 +195,12 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
//#import-context
|
//#import-context
|
||||||
|
|
||||||
val first = system.actorOf(Props(new FirstActor), name = "first")
|
val first = system.actorOf(Props(classOf[FirstActor], this), name = "first")
|
||||||
system.stop(first)
|
system.stop(first)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"creating actor with AkkaSpec.actorOf" in {
|
"creating actor with system.actorOf" in {
|
||||||
val myActor = system.actorOf(Props[MyActor])
|
val myActor = system.actorOf(Props[MyActor])
|
||||||
|
|
||||||
// testing the actor
|
// testing the actor
|
||||||
|
|
@ -183,44 +226,99 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
|
|
||||||
"creating actor with constructor" in {
|
"creating actor with constructor" in {
|
||||||
class MyActor(arg: String) extends Actor {
|
|
||||||
def receive = { case _ ⇒ () }
|
|
||||||
}
|
|
||||||
|
|
||||||
//#creating-constructor
|
//#creating-constructor
|
||||||
// allows passing in arguments to the MyActor constructor
|
// allows passing in arguments to the MyActor constructor
|
||||||
val myActor = system.actorOf(Props(new MyActor("...")), name = "myactor")
|
val myActor = system.actorOf(Props[MyActor], name = "myactor")
|
||||||
//#creating-constructor
|
//#creating-constructor
|
||||||
|
|
||||||
system.stop(myActor)
|
system.stop(myActor)
|
||||||
}
|
}
|
||||||
|
|
||||||
"creating a Props config" in {
|
"creating a Props config" in {
|
||||||
//#creating-props-config
|
//#creating-props
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
val props1 = Props.empty
|
|
||||||
val props2 = Props[MyActor]
|
val props1 = Props[MyActor]
|
||||||
val props3 = Props(new MyActor)
|
val props3 = Props(classOf[ActorWithArgs], "arg")
|
||||||
|
//#creating-props
|
||||||
|
|
||||||
|
//#creating-props-deprecated
|
||||||
|
// DEPRECATED: encourages to close over enclosing class
|
||||||
val props4 = Props(
|
val props4 = Props(
|
||||||
creator = { () ⇒ new MyActor },
|
creator = { () ⇒ new MyActor },
|
||||||
dispatcher = "my-dispatcher")
|
dispatcher = "my-dispatcher")
|
||||||
|
|
||||||
|
// DEPRECATED: encourages to close over enclosing class
|
||||||
val props5 = props1.withCreator(new MyActor)
|
val props5 = props1.withCreator(new MyActor)
|
||||||
val props6 = props5.withDispatcher("my-dispatcher")
|
|
||||||
//#creating-props-config
|
// DEPRECATED: encourages to close over enclosing class
|
||||||
|
val props6 = Props(new MyActor)
|
||||||
|
|
||||||
|
// DEPRECATED due to duplicate functionality with Props.apply()
|
||||||
|
val props7 = props1.withCreator(classOf[MyActor])
|
||||||
|
//#creating-props-deprecated
|
||||||
}
|
}
|
||||||
|
|
||||||
"creating actor with Props" in {
|
"creating actor with Props" in {
|
||||||
//#creating-props
|
//#system-actorOf
|
||||||
import akka.actor.Props
|
import akka.actor.ActorSystem
|
||||||
val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"),
|
|
||||||
name = "myactor2")
|
|
||||||
//#creating-props
|
|
||||||
|
|
||||||
system.stop(myActor)
|
// ActorSystem is a heavy object: create only one per application
|
||||||
|
val system = ActorSystem("mySystem")
|
||||||
|
val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), "myactor2")
|
||||||
|
//#system-actorOf
|
||||||
|
system.shutdown()
|
||||||
|
}
|
||||||
|
|
||||||
|
"creating actor with IndirectActorProducer" in {
|
||||||
|
class Echo(name: String) extends Actor {
|
||||||
|
def receive = {
|
||||||
|
case n: Int ⇒ sender ! name
|
||||||
|
case message ⇒
|
||||||
|
val target = testActor
|
||||||
|
//#forward
|
||||||
|
target forward message
|
||||||
|
//#forward
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
val a: { def actorRef: ActorRef } = new AnyRef {
|
||||||
|
val applicationContext = this
|
||||||
|
|
||||||
|
//#creating-indirectly
|
||||||
|
import akka.actor.IndirectActorProducer
|
||||||
|
|
||||||
|
class DependencyInjector(applicationContext: AnyRef, beanName: String)
|
||||||
|
extends IndirectActorProducer {
|
||||||
|
|
||||||
|
override def actorClass = classOf[Actor]
|
||||||
|
override def produce =
|
||||||
|
//#obtain-fresh-Actor-instance-from-DI-framework
|
||||||
|
new Echo(beanName)
|
||||||
|
|
||||||
|
def this(beanName: String) = this("", beanName)
|
||||||
|
//#obtain-fresh-Actor-instance-from-DI-framework
|
||||||
|
}
|
||||||
|
|
||||||
|
val actorRef = system.actorOf(
|
||||||
|
Props(classOf[DependencyInjector], applicationContext, "hello"),
|
||||||
|
"helloBean")
|
||||||
|
//#creating-indirectly
|
||||||
|
}
|
||||||
|
val actorRef = a.actorRef
|
||||||
|
|
||||||
|
val message = 42
|
||||||
|
implicit val self = testActor
|
||||||
|
//#tell
|
||||||
|
actorRef ! message
|
||||||
|
//#tell
|
||||||
|
expectMsg("hello")
|
||||||
|
actorRef ! "huhu"
|
||||||
|
expectMsg("huhu")
|
||||||
}
|
}
|
||||||
|
|
||||||
"using implicit timeout" in {
|
"using implicit timeout" in {
|
||||||
val myActor = system.actorOf(Props(new FirstActor))
|
val myActor = system.actorOf(Props[FirstActor])
|
||||||
//#using-implicit-timeout
|
//#using-implicit-timeout
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
|
|
@ -233,7 +331,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
|
|
||||||
"using explicit timeout" in {
|
"using explicit timeout" in {
|
||||||
val myActor = system.actorOf(Props(new FirstActor))
|
val myActor = system.actorOf(Props[FirstActor])
|
||||||
//#using-explicit-timeout
|
//#using-explicit-timeout
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
|
@ -262,7 +360,6 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
//#receive-timeout
|
//#receive-timeout
|
||||||
}
|
}
|
||||||
|
|
||||||
"using hot-swap" in {
|
|
||||||
//#hot-swap-actor
|
//#hot-swap-actor
|
||||||
class HotSwapActor extends Actor {
|
class HotSwapActor extends Actor {
|
||||||
import context._
|
import context._
|
||||||
|
|
@ -283,7 +380,8 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
//#hot-swap-actor
|
//#hot-swap-actor
|
||||||
|
|
||||||
val actor = system.actorOf(Props(new HotSwapActor), name = "hot")
|
"using hot-swap" in {
|
||||||
|
val actor = system.actorOf(Props(classOf[HotSwapActor], this), name = "hot")
|
||||||
}
|
}
|
||||||
|
|
||||||
"using Stash" in {
|
"using Stash" in {
|
||||||
|
|
@ -307,6 +405,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
|
|
||||||
"using watch" in {
|
"using watch" in {
|
||||||
|
new AnyRef {
|
||||||
//#watch
|
//#watch
|
||||||
import akka.actor.{ Actor, Props, Terminated }
|
import akka.actor.{ Actor, Props, Terminated }
|
||||||
|
|
||||||
|
|
@ -316,18 +415,40 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
var lastSender = system.deadLetters
|
var lastSender = system.deadLetters
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case "kill" ⇒ context.stop(child); lastSender = sender
|
case "kill" ⇒
|
||||||
|
context.stop(child); lastSender = sender
|
||||||
case Terminated(`child`) ⇒ lastSender ! "finished"
|
case Terminated(`child`) ⇒ lastSender ! "finished"
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//#watch
|
//#watch
|
||||||
val a = system.actorOf(Props(new WatchActor))
|
val a = system.actorOf(Props(classOf[WatchActor], this))
|
||||||
implicit val sender = testActor
|
implicit val sender = testActor
|
||||||
a ! "kill"
|
a ! "kill"
|
||||||
expectMsg("finished")
|
expectMsg("finished")
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"demonstrate ActorSelection" in {
|
||||||
|
val context = system
|
||||||
|
//#selection-local
|
||||||
|
// will look up this absolute path
|
||||||
|
context.actorSelection("/user/serviceA/aggregator")
|
||||||
|
// will look up sibling beneath same supervisor
|
||||||
|
context.actorSelection("../joe")
|
||||||
|
//#selection-local
|
||||||
|
//#selection-wildcard
|
||||||
|
// will look all children to serviceB with names starting with worker
|
||||||
|
context.actorSelection("/user/serviceB/worker*")
|
||||||
|
// will look up all siblings beneath same supervisor
|
||||||
|
context.actorSelection("../*")
|
||||||
|
//#selection-wildcard
|
||||||
|
//#selection-remote
|
||||||
|
context.actorSelection("akka.tcp://app@otherhost:1234/user/serviceB")
|
||||||
|
//#selection-remote
|
||||||
|
}
|
||||||
|
|
||||||
"using Identify" in {
|
"using Identify" in {
|
||||||
|
new AnyRef {
|
||||||
//#identify
|
//#identify
|
||||||
import akka.actor.{ Actor, Props, Identify, ActorIdentity, Terminated }
|
import akka.actor.{ Actor, Props, Identify, ActorIdentity, Terminated }
|
||||||
|
|
||||||
|
|
@ -349,14 +470,13 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
//#identify
|
//#identify
|
||||||
|
|
||||||
val a = system.actorOf(Props(new Actor {
|
val a = system.actorOf(Props.empty)
|
||||||
def receive = Actor.emptyBehavior
|
val b = system.actorOf(Props(classOf[Follower], this))
|
||||||
}))
|
|
||||||
val b = system.actorOf(Props(new Follower))
|
|
||||||
watch(b)
|
watch(b)
|
||||||
system.stop(a)
|
system.stop(a)
|
||||||
expectMsgType[akka.actor.Terminated].actor must be === b
|
expectMsgType[akka.actor.Terminated].actor must be === b
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
"using pattern gracefulStop" in {
|
"using pattern gracefulStop" in {
|
||||||
val actorRef = system.actorOf(Props[MyActor])
|
val actorRef = system.actorOf(Props[MyActor])
|
||||||
|
|
@ -397,8 +517,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
//#ask-pipeTo
|
//#ask-pipeTo
|
||||||
}
|
}
|
||||||
|
|
||||||
"replying with own or other sender" in {
|
class Replier extends Actor {
|
||||||
val actor = system.actorOf(Props(new Actor {
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case ref: ActorRef ⇒
|
case ref: ActorRef ⇒
|
||||||
//#reply-with-sender
|
//#reply-with-sender
|
||||||
|
|
@ -410,7 +529,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
sender ! x // replies will go to this actor
|
sender ! x // replies will go to this actor
|
||||||
//#reply-without-sender
|
//#reply-without-sender
|
||||||
}
|
}
|
||||||
}))
|
}
|
||||||
|
|
||||||
|
"replying with own or other sender" in {
|
||||||
|
val actor = system.actorOf(Props(classOf[Replier], this))
|
||||||
implicit val me = testActor
|
implicit val me = testActor
|
||||||
actor ! 42
|
actor ! 42
|
||||||
expectMsg(42)
|
expectMsg(42)
|
||||||
|
|
@ -430,7 +552,6 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
"using ComposableActor" in {
|
|
||||||
//#receive-orElse2
|
//#receive-orElse2
|
||||||
class PartialFunctionBuilder[A, B] {
|
class PartialFunctionBuilder[A, B] {
|
||||||
import scala.collection.immutable.Vector
|
import scala.collection.immutable.Vector
|
||||||
|
|
@ -474,7 +595,8 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
//#receive-orElse2
|
//#receive-orElse2
|
||||||
|
|
||||||
val composed = system.actorOf(Props(new MyComposableActor))
|
"using ComposableActor" in {
|
||||||
|
val composed = system.actorOf(Props(classOf[MyComposableActor], this))
|
||||||
implicit val me = testActor
|
implicit val me = testActor
|
||||||
composed ! "foo"
|
composed ! "foo"
|
||||||
expectMsg("foo received")
|
expectMsg("foo received")
|
||||||
|
|
|
||||||
|
|
@ -14,7 +14,6 @@ import scala.collection.immutable
|
||||||
|
|
||||||
class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
|
|
||||||
"simple finite state machine" must {
|
|
||||||
//#fsm-code-elided
|
//#fsm-code-elided
|
||||||
//#simple-imports
|
//#simple-imports
|
||||||
import akka.actor.{ Actor, ActorRef, FSM }
|
import akka.actor.{ Actor, ActorRef, FSM }
|
||||||
|
|
@ -189,6 +188,8 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
}
|
}
|
||||||
//#fsm-code-elided
|
//#fsm-code-elided
|
||||||
|
|
||||||
|
"simple finite state machine" must {
|
||||||
|
|
||||||
"demonstrate NullFunction" in {
|
"demonstrate NullFunction" in {
|
||||||
class A extends Actor with FSM[Int, Null] {
|
class A extends Actor with FSM[Int, Null] {
|
||||||
val SomeState = 0
|
val SomeState = 0
|
||||||
|
|
@ -199,7 +200,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
}
|
}
|
||||||
|
|
||||||
"batch correctly" in {
|
"batch correctly" in {
|
||||||
val buncher = system.actorOf(Props(new Buncher))
|
val buncher = system.actorOf(Props(classOf[Buncher], this))
|
||||||
buncher ! SetTarget(testActor)
|
buncher ! SetTarget(testActor)
|
||||||
buncher ! Queue(42)
|
buncher ! Queue(42)
|
||||||
buncher ! Queue(43)
|
buncher ! Queue(43)
|
||||||
|
|
@ -212,7 +213,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
}
|
}
|
||||||
|
|
||||||
"not batch if uninitialized" in {
|
"not batch if uninitialized" in {
|
||||||
val buncher = system.actorOf(Props(new Buncher))
|
val buncher = system.actorOf(Props(classOf[Buncher], this))
|
||||||
buncher ! Queue(42)
|
buncher ! Queue(42)
|
||||||
expectNoMsg
|
expectNoMsg
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -168,7 +168,7 @@ class CounterService extends Actor {
|
||||||
|
|
||||||
case Entry(k, v) if k == key && counter == None ⇒
|
case Entry(k, v) if k == key && counter == None ⇒
|
||||||
// Reply from Storage of the initial value, now we can create the Counter
|
// Reply from Storage of the initial value, now we can create the Counter
|
||||||
val c = context.actorOf(Props(new Counter(key, v)))
|
val c = context.actorOf(Props(classOf[Counter], key, v))
|
||||||
counter = Some(c)
|
counter = Some(c)
|
||||||
// Tell the counter to use current storage
|
// Tell the counter to use current storage
|
||||||
c ! UseStorage(storage)
|
c ! UseStorage(storage)
|
||||||
|
|
|
||||||
|
|
@ -38,13 +38,15 @@ class SchedulerDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
}
|
}
|
||||||
|
|
||||||
"schedule a recurring task" in {
|
"schedule a recurring task" in {
|
||||||
|
new AnyRef {
|
||||||
//#schedule-recurring
|
//#schedule-recurring
|
||||||
val Tick = "tick"
|
val Tick = "tick"
|
||||||
val tickActor = system.actorOf(Props(new Actor {
|
class TickActor extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case Tick ⇒ //Do something
|
case Tick ⇒ //Do something
|
||||||
}
|
}
|
||||||
}))
|
}
|
||||||
|
val tickActor = system.actorOf(Props(classOf[TickActor], this))
|
||||||
//Use system's dispatcher as ExecutionContext
|
//Use system's dispatcher as ExecutionContext
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
|
|
@ -61,4 +63,5 @@ class SchedulerDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
//#schedule-recurring
|
//#schedule-recurring
|
||||||
system.stop(tickActor)
|
system.stop(tickActor)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -44,8 +44,8 @@ object CustomRouteExample {
|
||||||
// example from a MicroKernel
|
// example from a MicroKernel
|
||||||
val system = ActorSystem("some-system")
|
val system = ActorSystem("some-system")
|
||||||
val producer = system.actorOf(Props[Producer1])
|
val producer = system.actorOf(Props[Producer1])
|
||||||
val mediator = system.actorOf(Props(new Transformer(producer)))
|
val mediator = system.actorOf(Props(classOf[Transformer], producer))
|
||||||
val consumer = system.actorOf(Props(new Consumer3(mediator)))
|
val consumer = system.actorOf(Props(classOf[Consumer3], mediator))
|
||||||
CamelExtension(system).context.addRoutes(new CustomRouteBuilder)
|
CamelExtension(system).context.addRoutes(new CustomRouteBuilder)
|
||||||
//#CustomRouteExample
|
//#CustomRouteExample
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -43,8 +43,8 @@ object HttpExample {
|
||||||
// to your boot class.
|
// to your boot class.
|
||||||
val system = ActorSystem("some-system")
|
val system = ActorSystem("some-system")
|
||||||
val httpTransformer = system.actorOf(Props[HttpTransformer])
|
val httpTransformer = system.actorOf(Props[HttpTransformer])
|
||||||
val httpProducer = system.actorOf(Props(new HttpProducer(httpTransformer)))
|
val httpProducer = system.actorOf(Props(classOf[HttpProducer], httpTransformer))
|
||||||
val httpConsumer = system.actorOf(Props(new HttpConsumer(httpProducer)))
|
val httpConsumer = system.actorOf(Props(classOf[HttpConsumer], httpProducer))
|
||||||
//#HttpExample
|
//#HttpExample
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -45,8 +45,8 @@ object Producers {
|
||||||
}
|
}
|
||||||
val system = ActorSystem("some-system")
|
val system = ActorSystem("some-system")
|
||||||
val receiver = system.actorOf(Props[ResponseReceiver])
|
val receiver = system.actorOf(Props[ResponseReceiver])
|
||||||
val forwardResponse = system.actorOf(Props(
|
val forwardResponse = system.actorOf(
|
||||||
new Forwarder("http://localhost:8080/news/akka", receiver)))
|
Props(classOf[Forwarder], this, "http://localhost:8080/news/akka", receiver))
|
||||||
// the Forwarder sends out a request to the web page and forwards the response to
|
// the Forwarder sends out a request to the web page and forwards the response to
|
||||||
// the ResponseReceiver
|
// the ResponseReceiver
|
||||||
forwardResponse ! "some request"
|
forwardResponse ! "some request"
|
||||||
|
|
@ -81,7 +81,7 @@ object Producers {
|
||||||
}
|
}
|
||||||
|
|
||||||
val system = ActorSystem("some-system")
|
val system = ActorSystem("some-system")
|
||||||
val producer = system.actorOf(Props(new OnewaySender("activemq:FOO.BAR")))
|
val producer = system.actorOf(Props(classOf[OnewaySender], this, "activemq:FOO.BAR"))
|
||||||
producer ! "Some message"
|
producer ! "Some message"
|
||||||
//#Oneway
|
//#Oneway
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,7 +1,6 @@
|
||||||
package docs.camel
|
package docs.camel
|
||||||
|
|
||||||
object PublishSubscribe {
|
object PublishSubscribe {
|
||||||
{
|
|
||||||
//#PubSub
|
//#PubSub
|
||||||
import akka.actor.{ Actor, ActorRef, ActorSystem, Props }
|
import akka.actor.{ Actor, ActorRef, ActorSystem, Props }
|
||||||
import akka.camel.{ Producer, CamelMessage, Consumer }
|
import akka.camel.{ Producer, CamelMessage, Consumer }
|
||||||
|
|
@ -37,11 +36,9 @@ object PublishSubscribe {
|
||||||
// Setup publish/subscribe example
|
// Setup publish/subscribe example
|
||||||
val system = ActorSystem("some-system")
|
val system = ActorSystem("some-system")
|
||||||
val jmsUri = "jms:topic:test"
|
val jmsUri = "jms:topic:test"
|
||||||
val jmsSubscriber1 = system.actorOf(Props(new Subscriber("jms-subscriber-1", jmsUri)))
|
val jmsSubscriber1 = system.actorOf(Props(classOf[Subscriber], "jms-subscriber-1", jmsUri))
|
||||||
val jmsSubscriber2 = system.actorOf(Props(new Subscriber("jms-subscriber-2", jmsUri)))
|
val jmsSubscriber2 = system.actorOf(Props(classOf[Subscriber], "jms-subscriber-2", jmsUri))
|
||||||
val jmsPublisher = system.actorOf(Props(new Publisher("jms-publisher", jmsUri)))
|
val jmsPublisher = system.actorOf(Props(classOf[Publisher], "jms-publisher", jmsUri))
|
||||||
val jmsPublisherBridge = system.actorOf(Props(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/jms", jmsPublisher)))
|
val jmsPublisherBridge = system.actorOf(Props(classOf[PublisherBridge], "jetty:http://0.0.0.0:8877/camel/pub/jms", jmsPublisher))
|
||||||
//#PubSub
|
//#PubSub
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -210,11 +210,11 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
|
||||||
}
|
}
|
||||||
|
|
||||||
"defining priority dispatcher" in {
|
"defining priority dispatcher" in {
|
||||||
|
new AnyRef {
|
||||||
//#prio-dispatcher
|
//#prio-dispatcher
|
||||||
|
|
||||||
// We create a new Actor that just prints out what it processes
|
// We create a new Actor that just prints out what it processes
|
||||||
val a = system.actorOf(
|
class Logger extends Actor {
|
||||||
Props(new Actor {
|
|
||||||
val log: LoggingAdapter = Logging(context.system, this)
|
val log: LoggingAdapter = Logging(context.system, this)
|
||||||
|
|
||||||
self ! 'lowpriority
|
self ! 'lowpriority
|
||||||
|
|
@ -229,23 +229,25 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
|
||||||
def receive = {
|
def receive = {
|
||||||
case x ⇒ log.info(x.toString)
|
case x ⇒ log.info(x.toString)
|
||||||
}
|
}
|
||||||
}).withDispatcher("prio-dispatcher"))
|
}
|
||||||
|
val a = system.actorOf(Props(classOf[Logger], this).withDispatcher("prio-dispatcher"))
|
||||||
|
|
||||||
/*
|
/*
|
||||||
Logs:
|
* Logs:
|
||||||
'highpriority
|
* 'highpriority
|
||||||
'highpriority
|
* 'highpriority
|
||||||
'pigdog
|
* 'pigdog
|
||||||
'pigdog2
|
* 'pigdog2
|
||||||
'pigdog3
|
* 'pigdog3
|
||||||
'lowpriority
|
* 'lowpriority
|
||||||
'lowpriority
|
* 'lowpriority
|
||||||
*/
|
*/
|
||||||
//#prio-dispatcher
|
//#prio-dispatcher
|
||||||
|
|
||||||
watch(a)
|
watch(a)
|
||||||
expectMsgPF() { case Terminated(`a`) ⇒ () }
|
expectMsgPF() { case Terminated(`a`) ⇒ () }
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
"defining balancing dispatcher" in {
|
"defining balancing dispatcher" in {
|
||||||
val dispatcher = system.dispatchers.lookup("my-balancing-dispatcher")
|
val dispatcher = system.dispatchers.lookup("my-balancing-dispatcher")
|
||||||
|
|
|
||||||
|
|
@ -72,22 +72,25 @@ class LoggingDocSpec extends AkkaSpec {
|
||||||
import LoggingDocSpec.MyActor
|
import LoggingDocSpec.MyActor
|
||||||
|
|
||||||
"use a logging actor" in {
|
"use a logging actor" in {
|
||||||
val myActor = system.actorOf(Props(new MyActor))
|
val myActor = system.actorOf(Props[MyActor])
|
||||||
myActor ! "test"
|
myActor ! "test"
|
||||||
}
|
}
|
||||||
|
|
||||||
"allow registration to dead letters" in {
|
"allow registration to dead letters" in {
|
||||||
|
new AnyRef {
|
||||||
//#deadletters
|
//#deadletters
|
||||||
import akka.actor.{ Actor, DeadLetter, Props }
|
import akka.actor.{ Actor, DeadLetter, Props }
|
||||||
|
|
||||||
val listener = system.actorOf(Props(new Actor {
|
class Listener extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case d: DeadLetter ⇒ println(d)
|
case d: DeadLetter ⇒ println(d)
|
||||||
}
|
}
|
||||||
}))
|
}
|
||||||
|
val listener = system.actorOf(Props(classOf[Listener], this))
|
||||||
system.eventStream.subscribe(listener, classOf[DeadLetter])
|
system.eventStream.subscribe(listener, classOf[DeadLetter])
|
||||||
//#deadletters
|
//#deadletters
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
"demonstrate logging more arguments" in {
|
"demonstrate logging more arguments" in {
|
||||||
//#array
|
//#array
|
||||||
|
|
|
||||||
|
|
@ -237,6 +237,6 @@ case class OKResponse(body: ByteString, keepAlive: Boolean)
|
||||||
object Main extends App {
|
object Main extends App {
|
||||||
val port = Option(System.getenv("PORT")) map (_.toInt) getOrElse 8080
|
val port = Option(System.getenv("PORT")) map (_.toInt) getOrElse 8080
|
||||||
val system = ActorSystem()
|
val system = ActorSystem()
|
||||||
val server = system.actorOf(Props(new HttpServer(port)))
|
val server = system.actorOf(Props(classOf[HttpServer], port))
|
||||||
}
|
}
|
||||||
//#main
|
//#main
|
||||||
|
|
|
||||||
|
|
@ -167,6 +167,32 @@ class PipelinesDocSpec extends AkkaSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
"demonstrate management port and context" in {
|
"demonstrate management port and context" in {
|
||||||
|
import TickGenerator.Tick
|
||||||
|
val proc = system.actorOf(Props(classOf[P], this, testActor, testActor), "processor")
|
||||||
|
expectMsgType[Tick]
|
||||||
|
proc ! msg
|
||||||
|
val encoded = expectMsgType[ByteString]
|
||||||
|
proc ! encoded
|
||||||
|
val decoded = expectMsgType[Message]
|
||||||
|
decoded must be === msg
|
||||||
|
|
||||||
|
within(1.5.seconds, 3.seconds) {
|
||||||
|
expectMsgType[Tick]
|
||||||
|
expectMsgType[Tick]
|
||||||
|
}
|
||||||
|
EventFilter[RuntimeException]("FAIL!", occurrences = 1) intercept {
|
||||||
|
proc ! "fail!"
|
||||||
|
}
|
||||||
|
within(1.5.seconds, 3.seconds) {
|
||||||
|
expectMsgType[Tick]
|
||||||
|
expectMsgType[Tick]
|
||||||
|
proc ! PoisonPill
|
||||||
|
expectNoMsg
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
//#actor
|
//#actor
|
||||||
class Processor(cmds: ActorRef, evts: ActorRef) extends Actor {
|
class Processor(cmds: ActorRef, evts: ActorRef) extends Actor {
|
||||||
|
|
||||||
|
|
@ -192,34 +218,10 @@ class PipelinesDocSpec extends AkkaSpec {
|
||||||
}
|
}
|
||||||
//#actor
|
//#actor
|
||||||
|
|
||||||
import TickGenerator.Tick
|
class P(cmds: ActorRef, evts: ActorRef) extends Processor(cmds, evts) {
|
||||||
val proc = system.actorOf(Props(new Processor(testActor, testActor) {
|
|
||||||
override def receive = ({
|
override def receive = ({
|
||||||
case "fail!" ⇒ throw new RuntimeException("FAIL!")
|
case "fail!" ⇒ throw new RuntimeException("FAIL!")
|
||||||
}: Receive) orElse super.receive
|
}: Receive) orElse super.receive
|
||||||
}), "processor")
|
|
||||||
expectMsgType[Tick]
|
|
||||||
proc ! msg
|
|
||||||
val encoded = expectMsgType[ByteString]
|
|
||||||
proc ! encoded
|
|
||||||
val decoded = expectMsgType[Message]
|
|
||||||
decoded must be === msg
|
|
||||||
|
|
||||||
within(1.5.seconds, 3.seconds) {
|
|
||||||
expectMsgType[Tick]
|
|
||||||
expectMsgType[Tick]
|
|
||||||
}
|
|
||||||
EventFilter[RuntimeException]("FAIL!", occurrences = 1) intercept {
|
|
||||||
proc ! "fail!"
|
|
||||||
}
|
|
||||||
within(1.5.seconds, 3.seconds) {
|
|
||||||
expectMsgType[Tick]
|
|
||||||
expectMsgType[Tick]
|
|
||||||
proc ! PoisonPill
|
|
||||||
expectNoMsg
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -88,12 +88,12 @@ class SchedulerPatternSpec extends AkkaSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
"send periodic ticks from the constructor" taggedAs TimingTest in {
|
"send periodic ticks from the constructor" taggedAs TimingTest in {
|
||||||
testSchedule(system.actorOf(Props(new ScheduleInConstructor(testActor))),
|
testSchedule(system.actorOf(Props(classOf[ScheduleInConstructor], testActor)),
|
||||||
3000 millis, 2000 millis)
|
3000 millis, 2000 millis)
|
||||||
}
|
}
|
||||||
|
|
||||||
"send ticks from the preStart and receive" taggedAs TimingTest in {
|
"send ticks from the preStart and receive" taggedAs TimingTest in {
|
||||||
testSchedule(system.actorOf(Props(new ScheduleInConstructor(testActor))),
|
testSchedule(system.actorOf(Props(classOf[ScheduleInConstructor], testActor)),
|
||||||
3000 millis, 2500 millis)
|
3000 millis, 2500 millis)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -34,15 +34,15 @@ class TestKitUsageSpec
|
||||||
with WordSpec with ShouldMatchers with BeforeAndAfterAll {
|
with WordSpec with ShouldMatchers with BeforeAndAfterAll {
|
||||||
import TestKitUsageSpec._
|
import TestKitUsageSpec._
|
||||||
|
|
||||||
val echoRef = system.actorOf(Props(new EchoActor))
|
val echoRef = system.actorOf(Props[EchoActor])
|
||||||
val forwardRef = system.actorOf(Props(new ForwardingActor(testActor)))
|
val forwardRef = system.actorOf(Props(classOf[ForwardingActor], testActor))
|
||||||
val filterRef = system.actorOf(Props(new FilteringActor(testActor)))
|
val filterRef = system.actorOf(Props(classOf[FilteringActor], testActor))
|
||||||
val randomHead = Random.nextInt(6)
|
val randomHead = Random.nextInt(6)
|
||||||
val randomTail = Random.nextInt(10)
|
val randomTail = Random.nextInt(10)
|
||||||
val headList = immutable.Seq().padTo(randomHead, "0")
|
val headList = immutable.Seq().padTo(randomHead, "0")
|
||||||
val tailList = immutable.Seq().padTo(randomTail, "1")
|
val tailList = immutable.Seq().padTo(randomTail, "1")
|
||||||
val seqRef =
|
val seqRef =
|
||||||
system.actorOf(Props(new SequencingActor(testActor, headList, tailList)))
|
system.actorOf(Props(classOf[SequencingActor], testActor, headList, tailList))
|
||||||
|
|
||||||
override def afterAll {
|
override def afterAll {
|
||||||
system.shutdown()
|
system.shutdown()
|
||||||
|
|
|
||||||
|
|
@ -208,9 +208,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
|
|
||||||
"demonstrate probe watch" in {
|
"demonstrate probe watch" in {
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
val target = system.actorOf(Props(new Actor {
|
val target = system.actorOf(Props.empty)
|
||||||
def receive = Actor.emptyBehavior
|
|
||||||
}))
|
|
||||||
//#test-probe-watch
|
//#test-probe-watch
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
probe watch target
|
probe watch target
|
||||||
|
|
@ -237,7 +235,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
//#test-probe-forward
|
//#test-probe-forward
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val source = system.actorOf(Props(new Source(probe.ref)))
|
val source = system.actorOf(Props(classOf[Source], probe.ref))
|
||||||
val dest = system.actorOf(Props[Destination])
|
val dest = system.actorOf(Props[Destination])
|
||||||
source ! "start"
|
source ! "start"
|
||||||
probe.expectMsg("work")
|
probe.expectMsg("work")
|
||||||
|
|
|
||||||
|
|
@ -199,6 +199,7 @@ class TransactorDocSpec extends AkkaSpec {
|
||||||
|
|
||||||
val system = ActorSystem("transactors")
|
val system = ActorSystem("transactors")
|
||||||
|
|
||||||
|
// FIXME, or remove the whole transactor module, srsly
|
||||||
lazy val underlyingCounter = new Counter
|
lazy val underlyingCounter = new Counter
|
||||||
val counter = system.actorOf(Props(underlyingCounter), name = "counter")
|
val counter = system.actorOf(Props(underlyingCounter), name = "counter")
|
||||||
val coordinated = Coordinated()(Timeout(5 seconds))
|
val coordinated = Coordinated()(Timeout(5 seconds))
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,6 @@
|
||||||
package docs.zeromq
|
package docs.zeromq
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.actor.{ Actor, Props }
|
import akka.actor.{ Actor, Props }
|
||||||
import akka.util.ByteString
|
import akka.util.ByteString
|
||||||
|
|
@ -12,6 +11,7 @@ import akka.testkit._
|
||||||
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind }
|
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind }
|
||||||
import java.text.SimpleDateFormat
|
import java.text.SimpleDateFormat
|
||||||
import java.util.Date
|
import java.util.Date
|
||||||
|
import akka.actor.ActorRef
|
||||||
|
|
||||||
object ZeromqDocSpec {
|
object ZeromqDocSpec {
|
||||||
|
|
||||||
|
|
@ -122,18 +122,25 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
|
||||||
Bind("tcp://127.0.0.1:21231"))
|
Bind("tcp://127.0.0.1:21231"))
|
||||||
//#pub-socket
|
//#pub-socket
|
||||||
|
|
||||||
|
import akka.zeromq._
|
||||||
|
val sub: { def subSocket: ActorRef; def listener: ActorRef } = new AnyRef {
|
||||||
//#sub-socket
|
//#sub-socket
|
||||||
import akka.zeromq._
|
import akka.zeromq._
|
||||||
val listener = system.actorOf(Props(new Actor {
|
|
||||||
|
class Listener extends Actor {
|
||||||
def receive: Receive = {
|
def receive: Receive = {
|
||||||
case Connecting ⇒ //...
|
case Connecting ⇒ //...
|
||||||
case m: ZMQMessage ⇒ //...
|
case m: ZMQMessage ⇒ //...
|
||||||
case _ ⇒ //...
|
case _ ⇒ //...
|
||||||
}
|
}
|
||||||
}))
|
}
|
||||||
|
|
||||||
|
val listener = system.actorOf(Props(classOf[Listener], this))
|
||||||
val subSocket = ZeroMQExtension(system).newSocket(SocketType.Sub,
|
val subSocket = ZeroMQExtension(system).newSocket(SocketType.Sub,
|
||||||
Listener(listener), Connect("tcp://127.0.0.1:21231"), SubscribeAll)
|
Listener(listener), Connect("tcp://127.0.0.1:21231"), SubscribeAll)
|
||||||
//#sub-socket
|
//#sub-socket
|
||||||
|
}
|
||||||
|
val listener = sub.listener
|
||||||
|
|
||||||
//#sub-topic-socket
|
//#sub-topic-socket
|
||||||
val subTopicSocket = ZeroMQExtension(system).newSocket(SocketType.Sub,
|
val subTopicSocket = ZeroMQExtension(system).newSocket(SocketType.Sub,
|
||||||
|
|
@ -149,7 +156,7 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
|
||||||
pubSocket ! ZMQMessage(ByteString("foo.bar"), ByteString(payload))
|
pubSocket ! ZMQMessage(ByteString("foo.bar"), ByteString(payload))
|
||||||
//#pub-topic
|
//#pub-topic
|
||||||
|
|
||||||
system.stop(subSocket)
|
system.stop(sub.subSocket)
|
||||||
system.stop(subTopicSocket)
|
system.stop(subTopicSocket)
|
||||||
|
|
||||||
//#high-watermark
|
//#high-watermark
|
||||||
|
|
|
||||||
|
|
@ -299,7 +299,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -886,7 +886,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -1882,7 +1882,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -2432,7 +2432,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -2841,7 +2841,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -3315,7 +3315,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -3858,7 +3858,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -4486,7 +4486,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -5034,26 +5034,24 @@ public final class RemoteProtocol {
|
||||||
public interface PropsProtocolOrBuilder
|
public interface PropsProtocolOrBuilder
|
||||||
extends com.google.protobuf.MessageOrBuilder {
|
extends com.google.protobuf.MessageOrBuilder {
|
||||||
|
|
||||||
// required string dispatcher = 1;
|
|
||||||
boolean hasDispatcher();
|
|
||||||
String getDispatcher();
|
|
||||||
|
|
||||||
// required .DeployProtocol deploy = 2;
|
// required .DeployProtocol deploy = 2;
|
||||||
boolean hasDeploy();
|
boolean hasDeploy();
|
||||||
akka.remote.RemoteProtocol.DeployProtocol getDeploy();
|
akka.remote.RemoteProtocol.DeployProtocol getDeploy();
|
||||||
akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder();
|
akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder();
|
||||||
|
|
||||||
// optional string fromClassCreator = 3;
|
// required string clazz = 3;
|
||||||
boolean hasFromClassCreator();
|
boolean hasClazz();
|
||||||
String getFromClassCreator();
|
String getClazz();
|
||||||
|
|
||||||
// optional bytes creator = 4;
|
// repeated bytes args = 4;
|
||||||
boolean hasCreator();
|
java.util.List<com.google.protobuf.ByteString> getArgsList();
|
||||||
com.google.protobuf.ByteString getCreator();
|
int getArgsCount();
|
||||||
|
com.google.protobuf.ByteString getArgs(int index);
|
||||||
|
|
||||||
// optional bytes routerConfig = 5;
|
// repeated string classes = 5;
|
||||||
boolean hasRouterConfig();
|
java.util.List<String> getClassesList();
|
||||||
com.google.protobuf.ByteString getRouterConfig();
|
int getClassesCount();
|
||||||
|
String getClasses(int index);
|
||||||
}
|
}
|
||||||
public static final class PropsProtocol extends
|
public static final class PropsProtocol extends
|
||||||
com.google.protobuf.GeneratedMessage
|
com.google.protobuf.GeneratedMessage
|
||||||
|
|
@ -5084,43 +5082,11 @@ public final class RemoteProtocol {
|
||||||
}
|
}
|
||||||
|
|
||||||
private int bitField0_;
|
private int bitField0_;
|
||||||
// required string dispatcher = 1;
|
|
||||||
public static final int DISPATCHER_FIELD_NUMBER = 1;
|
|
||||||
private java.lang.Object dispatcher_;
|
|
||||||
public boolean hasDispatcher() {
|
|
||||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
|
||||||
}
|
|
||||||
public String getDispatcher() {
|
|
||||||
java.lang.Object ref = dispatcher_;
|
|
||||||
if (ref instanceof String) {
|
|
||||||
return (String) ref;
|
|
||||||
} else {
|
|
||||||
com.google.protobuf.ByteString bs =
|
|
||||||
(com.google.protobuf.ByteString) ref;
|
|
||||||
String s = bs.toStringUtf8();
|
|
||||||
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
|
|
||||||
dispatcher_ = s;
|
|
||||||
}
|
|
||||||
return s;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
private com.google.protobuf.ByteString getDispatcherBytes() {
|
|
||||||
java.lang.Object ref = dispatcher_;
|
|
||||||
if (ref instanceof String) {
|
|
||||||
com.google.protobuf.ByteString b =
|
|
||||||
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
|
|
||||||
dispatcher_ = b;
|
|
||||||
return b;
|
|
||||||
} else {
|
|
||||||
return (com.google.protobuf.ByteString) ref;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// required .DeployProtocol deploy = 2;
|
// required .DeployProtocol deploy = 2;
|
||||||
public static final int DEPLOY_FIELD_NUMBER = 2;
|
public static final int DEPLOY_FIELD_NUMBER = 2;
|
||||||
private akka.remote.RemoteProtocol.DeployProtocol deploy_;
|
private akka.remote.RemoteProtocol.DeployProtocol deploy_;
|
||||||
public boolean hasDeploy() {
|
public boolean hasDeploy() {
|
||||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||||
}
|
}
|
||||||
public akka.remote.RemoteProtocol.DeployProtocol getDeploy() {
|
public akka.remote.RemoteProtocol.DeployProtocol getDeploy() {
|
||||||
return deploy_;
|
return deploy_;
|
||||||
|
|
@ -5129,14 +5095,14 @@ public final class RemoteProtocol {
|
||||||
return deploy_;
|
return deploy_;
|
||||||
}
|
}
|
||||||
|
|
||||||
// optional string fromClassCreator = 3;
|
// required string clazz = 3;
|
||||||
public static final int FROMCLASSCREATOR_FIELD_NUMBER = 3;
|
public static final int CLAZZ_FIELD_NUMBER = 3;
|
||||||
private java.lang.Object fromClassCreator_;
|
private java.lang.Object clazz_;
|
||||||
public boolean hasFromClassCreator() {
|
public boolean hasClazz() {
|
||||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||||
}
|
}
|
||||||
public String getFromClassCreator() {
|
public String getClazz() {
|
||||||
java.lang.Object ref = fromClassCreator_;
|
java.lang.Object ref = clazz_;
|
||||||
if (ref instanceof String) {
|
if (ref instanceof String) {
|
||||||
return (String) ref;
|
return (String) ref;
|
||||||
} else {
|
} else {
|
||||||
|
|
@ -5144,60 +5110,67 @@ public final class RemoteProtocol {
|
||||||
(com.google.protobuf.ByteString) ref;
|
(com.google.protobuf.ByteString) ref;
|
||||||
String s = bs.toStringUtf8();
|
String s = bs.toStringUtf8();
|
||||||
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
|
if (com.google.protobuf.Internal.isValidUtf8(bs)) {
|
||||||
fromClassCreator_ = s;
|
clazz_ = s;
|
||||||
}
|
}
|
||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
private com.google.protobuf.ByteString getFromClassCreatorBytes() {
|
private com.google.protobuf.ByteString getClazzBytes() {
|
||||||
java.lang.Object ref = fromClassCreator_;
|
java.lang.Object ref = clazz_;
|
||||||
if (ref instanceof String) {
|
if (ref instanceof String) {
|
||||||
com.google.protobuf.ByteString b =
|
com.google.protobuf.ByteString b =
|
||||||
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
|
com.google.protobuf.ByteString.copyFromUtf8((String) ref);
|
||||||
fromClassCreator_ = b;
|
clazz_ = b;
|
||||||
return b;
|
return b;
|
||||||
} else {
|
} else {
|
||||||
return (com.google.protobuf.ByteString) ref;
|
return (com.google.protobuf.ByteString) ref;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// optional bytes creator = 4;
|
// repeated bytes args = 4;
|
||||||
public static final int CREATOR_FIELD_NUMBER = 4;
|
public static final int ARGS_FIELD_NUMBER = 4;
|
||||||
private com.google.protobuf.ByteString creator_;
|
private java.util.List<com.google.protobuf.ByteString> args_;
|
||||||
public boolean hasCreator() {
|
public java.util.List<com.google.protobuf.ByteString>
|
||||||
return ((bitField0_ & 0x00000008) == 0x00000008);
|
getArgsList() {
|
||||||
|
return args_;
|
||||||
}
|
}
|
||||||
public com.google.protobuf.ByteString getCreator() {
|
public int getArgsCount() {
|
||||||
return creator_;
|
return args_.size();
|
||||||
|
}
|
||||||
|
public com.google.protobuf.ByteString getArgs(int index) {
|
||||||
|
return args_.get(index);
|
||||||
}
|
}
|
||||||
|
|
||||||
// optional bytes routerConfig = 5;
|
// repeated string classes = 5;
|
||||||
public static final int ROUTERCONFIG_FIELD_NUMBER = 5;
|
public static final int CLASSES_FIELD_NUMBER = 5;
|
||||||
private com.google.protobuf.ByteString routerConfig_;
|
private com.google.protobuf.LazyStringList classes_;
|
||||||
public boolean hasRouterConfig() {
|
public java.util.List<String>
|
||||||
return ((bitField0_ & 0x00000010) == 0x00000010);
|
getClassesList() {
|
||||||
|
return classes_;
|
||||||
}
|
}
|
||||||
public com.google.protobuf.ByteString getRouterConfig() {
|
public int getClassesCount() {
|
||||||
return routerConfig_;
|
return classes_.size();
|
||||||
|
}
|
||||||
|
public String getClasses(int index) {
|
||||||
|
return classes_.get(index);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initFields() {
|
private void initFields() {
|
||||||
dispatcher_ = "";
|
|
||||||
deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance();
|
deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance();
|
||||||
fromClassCreator_ = "";
|
clazz_ = "";
|
||||||
creator_ = com.google.protobuf.ByteString.EMPTY;
|
args_ = java.util.Collections.emptyList();;
|
||||||
routerConfig_ = com.google.protobuf.ByteString.EMPTY;
|
classes_ = com.google.protobuf.LazyStringArrayList.EMPTY;
|
||||||
}
|
}
|
||||||
private byte memoizedIsInitialized = -1;
|
private byte memoizedIsInitialized = -1;
|
||||||
public final boolean isInitialized() {
|
public final boolean isInitialized() {
|
||||||
byte isInitialized = memoizedIsInitialized;
|
byte isInitialized = memoizedIsInitialized;
|
||||||
if (isInitialized != -1) return isInitialized == 1;
|
if (isInitialized != -1) return isInitialized == 1;
|
||||||
|
|
||||||
if (!hasDispatcher()) {
|
if (!hasDeploy()) {
|
||||||
memoizedIsInitialized = 0;
|
memoizedIsInitialized = 0;
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
if (!hasDeploy()) {
|
if (!hasClazz()) {
|
||||||
memoizedIsInitialized = 0;
|
memoizedIsInitialized = 0;
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
@ -5213,19 +5186,16 @@ public final class RemoteProtocol {
|
||||||
throws java.io.IOException {
|
throws java.io.IOException {
|
||||||
getSerializedSize();
|
getSerializedSize();
|
||||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||||
output.writeBytes(1, getDispatcherBytes());
|
|
||||||
}
|
|
||||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
|
||||||
output.writeMessage(2, deploy_);
|
output.writeMessage(2, deploy_);
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||||
output.writeBytes(3, getFromClassCreatorBytes());
|
output.writeBytes(3, getClazzBytes());
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
for (int i = 0; i < args_.size(); i++) {
|
||||||
output.writeBytes(4, creator_);
|
output.writeBytes(4, args_.get(i));
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
for (int i = 0; i < classes_.size(); i++) {
|
||||||
output.writeBytes(5, routerConfig_);
|
output.writeBytes(5, classes_.getByteString(i));
|
||||||
}
|
}
|
||||||
getUnknownFields().writeTo(output);
|
getUnknownFields().writeTo(output);
|
||||||
}
|
}
|
||||||
|
|
@ -5238,23 +5208,29 @@ public final class RemoteProtocol {
|
||||||
size = 0;
|
size = 0;
|
||||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||||
size += com.google.protobuf.CodedOutputStream
|
size += com.google.protobuf.CodedOutputStream
|
||||||
.computeBytesSize(1, getDispatcherBytes());
|
.computeMessageSize(2, deploy_);
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||||
size += com.google.protobuf.CodedOutputStream
|
size += com.google.protobuf.CodedOutputStream
|
||||||
.computeMessageSize(2, deploy_);
|
.computeBytesSize(3, getClazzBytes());
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
{
|
||||||
size += com.google.protobuf.CodedOutputStream
|
int dataSize = 0;
|
||||||
.computeBytesSize(3, getFromClassCreatorBytes());
|
for (int i = 0; i < args_.size(); i++) {
|
||||||
|
dataSize += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeBytesSizeNoTag(args_.get(i));
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
size += dataSize;
|
||||||
size += com.google.protobuf.CodedOutputStream
|
size += 1 * getArgsList().size();
|
||||||
.computeBytesSize(4, creator_);
|
|
||||||
}
|
}
|
||||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
{
|
||||||
size += com.google.protobuf.CodedOutputStream
|
int dataSize = 0;
|
||||||
.computeBytesSize(5, routerConfig_);
|
for (int i = 0; i < classes_.size(); i++) {
|
||||||
|
dataSize += com.google.protobuf.CodedOutputStream
|
||||||
|
.computeBytesSizeNoTag(classes_.getByteString(i));
|
||||||
|
}
|
||||||
|
size += dataSize;
|
||||||
|
size += 1 * getClassesList().size();
|
||||||
}
|
}
|
||||||
size += getUnknownFields().getSerializedSize();
|
size += getUnknownFields().getSerializedSize();
|
||||||
memoizedSerializedSize = size;
|
memoizedSerializedSize = size;
|
||||||
|
|
@ -5366,7 +5342,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -5381,20 +5357,18 @@ public final class RemoteProtocol {
|
||||||
|
|
||||||
public Builder clear() {
|
public Builder clear() {
|
||||||
super.clear();
|
super.clear();
|
||||||
dispatcher_ = "";
|
|
||||||
bitField0_ = (bitField0_ & ~0x00000001);
|
|
||||||
if (deployBuilder_ == null) {
|
if (deployBuilder_ == null) {
|
||||||
deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance();
|
deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance();
|
||||||
} else {
|
} else {
|
||||||
deployBuilder_.clear();
|
deployBuilder_.clear();
|
||||||
}
|
}
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000001);
|
||||||
|
clazz_ = "";
|
||||||
bitField0_ = (bitField0_ & ~0x00000002);
|
bitField0_ = (bitField0_ & ~0x00000002);
|
||||||
fromClassCreator_ = "";
|
args_ = java.util.Collections.emptyList();;
|
||||||
bitField0_ = (bitField0_ & ~0x00000004);
|
bitField0_ = (bitField0_ & ~0x00000004);
|
||||||
creator_ = com.google.protobuf.ByteString.EMPTY;
|
classes_ = com.google.protobuf.LazyStringArrayList.EMPTY;
|
||||||
bitField0_ = (bitField0_ & ~0x00000008);
|
bitField0_ = (bitField0_ & ~0x00000008);
|
||||||
routerConfig_ = com.google.protobuf.ByteString.EMPTY;
|
|
||||||
bitField0_ = (bitField0_ & ~0x00000010);
|
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -5436,27 +5410,26 @@ public final class RemoteProtocol {
|
||||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||||
to_bitField0_ |= 0x00000001;
|
to_bitField0_ |= 0x00000001;
|
||||||
}
|
}
|
||||||
result.dispatcher_ = dispatcher_;
|
|
||||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
|
||||||
to_bitField0_ |= 0x00000002;
|
|
||||||
}
|
|
||||||
if (deployBuilder_ == null) {
|
if (deployBuilder_ == null) {
|
||||||
result.deploy_ = deploy_;
|
result.deploy_ = deploy_;
|
||||||
} else {
|
} else {
|
||||||
result.deploy_ = deployBuilder_.build();
|
result.deploy_ = deployBuilder_.build();
|
||||||
}
|
}
|
||||||
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
|
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||||
to_bitField0_ |= 0x00000004;
|
to_bitField0_ |= 0x00000002;
|
||||||
}
|
}
|
||||||
result.fromClassCreator_ = fromClassCreator_;
|
result.clazz_ = clazz_;
|
||||||
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
|
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||||
to_bitField0_ |= 0x00000008;
|
args_ = java.util.Collections.unmodifiableList(args_);
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000004);
|
||||||
}
|
}
|
||||||
result.creator_ = creator_;
|
result.args_ = args_;
|
||||||
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
|
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
||||||
to_bitField0_ |= 0x00000010;
|
classes_ = new com.google.protobuf.UnmodifiableLazyStringList(
|
||||||
|
classes_);
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000008);
|
||||||
}
|
}
|
||||||
result.routerConfig_ = routerConfig_;
|
result.classes_ = classes_;
|
||||||
result.bitField0_ = to_bitField0_;
|
result.bitField0_ = to_bitField0_;
|
||||||
onBuilt();
|
onBuilt();
|
||||||
return result;
|
return result;
|
||||||
|
|
@ -5473,31 +5446,42 @@ public final class RemoteProtocol {
|
||||||
|
|
||||||
public Builder mergeFrom(akka.remote.RemoteProtocol.PropsProtocol other) {
|
public Builder mergeFrom(akka.remote.RemoteProtocol.PropsProtocol other) {
|
||||||
if (other == akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance()) return this;
|
if (other == akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance()) return this;
|
||||||
if (other.hasDispatcher()) {
|
|
||||||
setDispatcher(other.getDispatcher());
|
|
||||||
}
|
|
||||||
if (other.hasDeploy()) {
|
if (other.hasDeploy()) {
|
||||||
mergeDeploy(other.getDeploy());
|
mergeDeploy(other.getDeploy());
|
||||||
}
|
}
|
||||||
if (other.hasFromClassCreator()) {
|
if (other.hasClazz()) {
|
||||||
setFromClassCreator(other.getFromClassCreator());
|
setClazz(other.getClazz());
|
||||||
}
|
}
|
||||||
if (other.hasCreator()) {
|
if (!other.args_.isEmpty()) {
|
||||||
setCreator(other.getCreator());
|
if (args_.isEmpty()) {
|
||||||
|
args_ = other.args_;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000004);
|
||||||
|
} else {
|
||||||
|
ensureArgsIsMutable();
|
||||||
|
args_.addAll(other.args_);
|
||||||
}
|
}
|
||||||
if (other.hasRouterConfig()) {
|
onChanged();
|
||||||
setRouterConfig(other.getRouterConfig());
|
}
|
||||||
|
if (!other.classes_.isEmpty()) {
|
||||||
|
if (classes_.isEmpty()) {
|
||||||
|
classes_ = other.classes_;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000008);
|
||||||
|
} else {
|
||||||
|
ensureClassesIsMutable();
|
||||||
|
classes_.addAll(other.classes_);
|
||||||
|
}
|
||||||
|
onChanged();
|
||||||
}
|
}
|
||||||
this.mergeUnknownFields(other.getUnknownFields());
|
this.mergeUnknownFields(other.getUnknownFields());
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public final boolean isInitialized() {
|
public final boolean isInitialized() {
|
||||||
if (!hasDispatcher()) {
|
if (!hasDeploy()) {
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
if (!hasDeploy()) {
|
if (!hasClazz()) {
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
@ -5531,11 +5515,6 @@ public final class RemoteProtocol {
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case 10: {
|
|
||||||
bitField0_ |= 0x00000001;
|
|
||||||
dispatcher_ = input.readBytes();
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case 18: {
|
case 18: {
|
||||||
akka.remote.RemoteProtocol.DeployProtocol.Builder subBuilder = akka.remote.RemoteProtocol.DeployProtocol.newBuilder();
|
akka.remote.RemoteProtocol.DeployProtocol.Builder subBuilder = akka.remote.RemoteProtocol.DeployProtocol.newBuilder();
|
||||||
if (hasDeploy()) {
|
if (hasDeploy()) {
|
||||||
|
|
@ -5546,18 +5525,18 @@ public final class RemoteProtocol {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case 26: {
|
case 26: {
|
||||||
bitField0_ |= 0x00000004;
|
bitField0_ |= 0x00000002;
|
||||||
fromClassCreator_ = input.readBytes();
|
clazz_ = input.readBytes();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case 34: {
|
case 34: {
|
||||||
bitField0_ |= 0x00000008;
|
ensureArgsIsMutable();
|
||||||
creator_ = input.readBytes();
|
args_.add(input.readBytes());
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
case 42: {
|
case 42: {
|
||||||
bitField0_ |= 0x00000010;
|
ensureClassesIsMutable();
|
||||||
routerConfig_ = input.readBytes();
|
classes_.add(input.readBytes());
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -5566,48 +5545,12 @@ public final class RemoteProtocol {
|
||||||
|
|
||||||
private int bitField0_;
|
private int bitField0_;
|
||||||
|
|
||||||
// required string dispatcher = 1;
|
|
||||||
private java.lang.Object dispatcher_ = "";
|
|
||||||
public boolean hasDispatcher() {
|
|
||||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
|
||||||
}
|
|
||||||
public String getDispatcher() {
|
|
||||||
java.lang.Object ref = dispatcher_;
|
|
||||||
if (!(ref instanceof String)) {
|
|
||||||
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
|
|
||||||
dispatcher_ = s;
|
|
||||||
return s;
|
|
||||||
} else {
|
|
||||||
return (String) ref;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
public Builder setDispatcher(String value) {
|
|
||||||
if (value == null) {
|
|
||||||
throw new NullPointerException();
|
|
||||||
}
|
|
||||||
bitField0_ |= 0x00000001;
|
|
||||||
dispatcher_ = value;
|
|
||||||
onChanged();
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
public Builder clearDispatcher() {
|
|
||||||
bitField0_ = (bitField0_ & ~0x00000001);
|
|
||||||
dispatcher_ = getDefaultInstance().getDispatcher();
|
|
||||||
onChanged();
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
void setDispatcher(com.google.protobuf.ByteString value) {
|
|
||||||
bitField0_ |= 0x00000001;
|
|
||||||
dispatcher_ = value;
|
|
||||||
onChanged();
|
|
||||||
}
|
|
||||||
|
|
||||||
// required .DeployProtocol deploy = 2;
|
// required .DeployProtocol deploy = 2;
|
||||||
private akka.remote.RemoteProtocol.DeployProtocol deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance();
|
private akka.remote.RemoteProtocol.DeployProtocol deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance();
|
||||||
private com.google.protobuf.SingleFieldBuilder<
|
private com.google.protobuf.SingleFieldBuilder<
|
||||||
akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder> deployBuilder_;
|
akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder> deployBuilder_;
|
||||||
public boolean hasDeploy() {
|
public boolean hasDeploy() {
|
||||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||||
}
|
}
|
||||||
public akka.remote.RemoteProtocol.DeployProtocol getDeploy() {
|
public akka.remote.RemoteProtocol.DeployProtocol getDeploy() {
|
||||||
if (deployBuilder_ == null) {
|
if (deployBuilder_ == null) {
|
||||||
|
|
@ -5626,7 +5569,7 @@ public final class RemoteProtocol {
|
||||||
} else {
|
} else {
|
||||||
deployBuilder_.setMessage(value);
|
deployBuilder_.setMessage(value);
|
||||||
}
|
}
|
||||||
bitField0_ |= 0x00000002;
|
bitField0_ |= 0x00000001;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
public Builder setDeploy(
|
public Builder setDeploy(
|
||||||
|
|
@ -5637,12 +5580,12 @@ public final class RemoteProtocol {
|
||||||
} else {
|
} else {
|
||||||
deployBuilder_.setMessage(builderForValue.build());
|
deployBuilder_.setMessage(builderForValue.build());
|
||||||
}
|
}
|
||||||
bitField0_ |= 0x00000002;
|
bitField0_ |= 0x00000001;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
public Builder mergeDeploy(akka.remote.RemoteProtocol.DeployProtocol value) {
|
public Builder mergeDeploy(akka.remote.RemoteProtocol.DeployProtocol value) {
|
||||||
if (deployBuilder_ == null) {
|
if (deployBuilder_ == null) {
|
||||||
if (((bitField0_ & 0x00000002) == 0x00000002) &&
|
if (((bitField0_ & 0x00000001) == 0x00000001) &&
|
||||||
deploy_ != akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance()) {
|
deploy_ != akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance()) {
|
||||||
deploy_ =
|
deploy_ =
|
||||||
akka.remote.RemoteProtocol.DeployProtocol.newBuilder(deploy_).mergeFrom(value).buildPartial();
|
akka.remote.RemoteProtocol.DeployProtocol.newBuilder(deploy_).mergeFrom(value).buildPartial();
|
||||||
|
|
@ -5653,7 +5596,7 @@ public final class RemoteProtocol {
|
||||||
} else {
|
} else {
|
||||||
deployBuilder_.mergeFrom(value);
|
deployBuilder_.mergeFrom(value);
|
||||||
}
|
}
|
||||||
bitField0_ |= 0x00000002;
|
bitField0_ |= 0x00000001;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
public Builder clearDeploy() {
|
public Builder clearDeploy() {
|
||||||
|
|
@ -5663,11 +5606,11 @@ public final class RemoteProtocol {
|
||||||
} else {
|
} else {
|
||||||
deployBuilder_.clear();
|
deployBuilder_.clear();
|
||||||
}
|
}
|
||||||
bitField0_ = (bitField0_ & ~0x00000002);
|
bitField0_ = (bitField0_ & ~0x00000001);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
public akka.remote.RemoteProtocol.DeployProtocol.Builder getDeployBuilder() {
|
public akka.remote.RemoteProtocol.DeployProtocol.Builder getDeployBuilder() {
|
||||||
bitField0_ |= 0x00000002;
|
bitField0_ |= 0x00000001;
|
||||||
onChanged();
|
onChanged();
|
||||||
return getDeployFieldBuilder().getBuilder();
|
return getDeployFieldBuilder().getBuilder();
|
||||||
}
|
}
|
||||||
|
|
@ -5692,89 +5635,148 @@ public final class RemoteProtocol {
|
||||||
return deployBuilder_;
|
return deployBuilder_;
|
||||||
}
|
}
|
||||||
|
|
||||||
// optional string fromClassCreator = 3;
|
// required string clazz = 3;
|
||||||
private java.lang.Object fromClassCreator_ = "";
|
private java.lang.Object clazz_ = "";
|
||||||
public boolean hasFromClassCreator() {
|
public boolean hasClazz() {
|
||||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||||
}
|
}
|
||||||
public String getFromClassCreator() {
|
public String getClazz() {
|
||||||
java.lang.Object ref = fromClassCreator_;
|
java.lang.Object ref = clazz_;
|
||||||
if (!(ref instanceof String)) {
|
if (!(ref instanceof String)) {
|
||||||
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
|
String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
|
||||||
fromClassCreator_ = s;
|
clazz_ = s;
|
||||||
return s;
|
return s;
|
||||||
} else {
|
} else {
|
||||||
return (String) ref;
|
return (String) ref;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
public Builder setFromClassCreator(String value) {
|
public Builder setClazz(String value) {
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
throw new NullPointerException();
|
throw new NullPointerException();
|
||||||
}
|
}
|
||||||
bitField0_ |= 0x00000004;
|
bitField0_ |= 0x00000002;
|
||||||
fromClassCreator_ = value;
|
clazz_ = value;
|
||||||
onChanged();
|
onChanged();
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
public Builder clearFromClassCreator() {
|
public Builder clearClazz() {
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000002);
|
||||||
|
clazz_ = getDefaultInstance().getClazz();
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
void setClazz(com.google.protobuf.ByteString value) {
|
||||||
|
bitField0_ |= 0x00000002;
|
||||||
|
clazz_ = value;
|
||||||
|
onChanged();
|
||||||
|
}
|
||||||
|
|
||||||
|
// repeated bytes args = 4;
|
||||||
|
private java.util.List<com.google.protobuf.ByteString> args_ = java.util.Collections.emptyList();;
|
||||||
|
private void ensureArgsIsMutable() {
|
||||||
|
if (!((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||||
|
args_ = new java.util.ArrayList<com.google.protobuf.ByteString>(args_);
|
||||||
|
bitField0_ |= 0x00000004;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
public java.util.List<com.google.protobuf.ByteString>
|
||||||
|
getArgsList() {
|
||||||
|
return java.util.Collections.unmodifiableList(args_);
|
||||||
|
}
|
||||||
|
public int getArgsCount() {
|
||||||
|
return args_.size();
|
||||||
|
}
|
||||||
|
public com.google.protobuf.ByteString getArgs(int index) {
|
||||||
|
return args_.get(index);
|
||||||
|
}
|
||||||
|
public Builder setArgs(
|
||||||
|
int index, com.google.protobuf.ByteString value) {
|
||||||
|
if (value == null) {
|
||||||
|
throw new NullPointerException();
|
||||||
|
}
|
||||||
|
ensureArgsIsMutable();
|
||||||
|
args_.set(index, value);
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder addArgs(com.google.protobuf.ByteString value) {
|
||||||
|
if (value == null) {
|
||||||
|
throw new NullPointerException();
|
||||||
|
}
|
||||||
|
ensureArgsIsMutable();
|
||||||
|
args_.add(value);
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder addAllArgs(
|
||||||
|
java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
|
||||||
|
ensureArgsIsMutable();
|
||||||
|
super.addAll(values, args_);
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearArgs() {
|
||||||
|
args_ = java.util.Collections.emptyList();;
|
||||||
bitField0_ = (bitField0_ & ~0x00000004);
|
bitField0_ = (bitField0_ & ~0x00000004);
|
||||||
fromClassCreator_ = getDefaultInstance().getFromClassCreator();
|
|
||||||
onChanged();
|
onChanged();
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
void setFromClassCreator(com.google.protobuf.ByteString value) {
|
|
||||||
bitField0_ |= 0x00000004;
|
|
||||||
fromClassCreator_ = value;
|
|
||||||
onChanged();
|
|
||||||
}
|
|
||||||
|
|
||||||
// optional bytes creator = 4;
|
// repeated string classes = 5;
|
||||||
private com.google.protobuf.ByteString creator_ = com.google.protobuf.ByteString.EMPTY;
|
private com.google.protobuf.LazyStringList classes_ = com.google.protobuf.LazyStringArrayList.EMPTY;
|
||||||
public boolean hasCreator() {
|
private void ensureClassesIsMutable() {
|
||||||
return ((bitField0_ & 0x00000008) == 0x00000008);
|
if (!((bitField0_ & 0x00000008) == 0x00000008)) {
|
||||||
}
|
classes_ = new com.google.protobuf.LazyStringArrayList(classes_);
|
||||||
public com.google.protobuf.ByteString getCreator() {
|
|
||||||
return creator_;
|
|
||||||
}
|
|
||||||
public Builder setCreator(com.google.protobuf.ByteString value) {
|
|
||||||
if (value == null) {
|
|
||||||
throw new NullPointerException();
|
|
||||||
}
|
|
||||||
bitField0_ |= 0x00000008;
|
bitField0_ |= 0x00000008;
|
||||||
creator_ = value;
|
|
||||||
onChanged();
|
|
||||||
return this;
|
|
||||||
}
|
}
|
||||||
public Builder clearCreator() {
|
|
||||||
bitField0_ = (bitField0_ & ~0x00000008);
|
|
||||||
creator_ = getDefaultInstance().getCreator();
|
|
||||||
onChanged();
|
|
||||||
return this;
|
|
||||||
}
|
}
|
||||||
|
public java.util.List<String>
|
||||||
// optional bytes routerConfig = 5;
|
getClassesList() {
|
||||||
private com.google.protobuf.ByteString routerConfig_ = com.google.protobuf.ByteString.EMPTY;
|
return java.util.Collections.unmodifiableList(classes_);
|
||||||
public boolean hasRouterConfig() {
|
|
||||||
return ((bitField0_ & 0x00000010) == 0x00000010);
|
|
||||||
}
|
}
|
||||||
public com.google.protobuf.ByteString getRouterConfig() {
|
public int getClassesCount() {
|
||||||
return routerConfig_;
|
return classes_.size();
|
||||||
}
|
}
|
||||||
public Builder setRouterConfig(com.google.protobuf.ByteString value) {
|
public String getClasses(int index) {
|
||||||
|
return classes_.get(index);
|
||||||
|
}
|
||||||
|
public Builder setClasses(
|
||||||
|
int index, String value) {
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
throw new NullPointerException();
|
throw new NullPointerException();
|
||||||
}
|
}
|
||||||
bitField0_ |= 0x00000010;
|
ensureClassesIsMutable();
|
||||||
routerConfig_ = value;
|
classes_.set(index, value);
|
||||||
onChanged();
|
onChanged();
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
public Builder clearRouterConfig() {
|
public Builder addClasses(String value) {
|
||||||
bitField0_ = (bitField0_ & ~0x00000010);
|
if (value == null) {
|
||||||
routerConfig_ = getDefaultInstance().getRouterConfig();
|
throw new NullPointerException();
|
||||||
|
}
|
||||||
|
ensureClassesIsMutable();
|
||||||
|
classes_.add(value);
|
||||||
onChanged();
|
onChanged();
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
public Builder addAllClasses(
|
||||||
|
java.lang.Iterable<String> values) {
|
||||||
|
ensureClassesIsMutable();
|
||||||
|
super.addAll(values, classes_);
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
public Builder clearClasses() {
|
||||||
|
classes_ = com.google.protobuf.LazyStringArrayList.EMPTY;
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000008);
|
||||||
|
onChanged();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
void addClasses(com.google.protobuf.ByteString value) {
|
||||||
|
ensureClassesIsMutable();
|
||||||
|
classes_.add(value);
|
||||||
|
onChanged();
|
||||||
|
}
|
||||||
|
|
||||||
// @@protoc_insertion_point(builder_scope:PropsProtocol)
|
// @@protoc_insertion_point(builder_scope:PropsProtocol)
|
||||||
}
|
}
|
||||||
|
|
@ -6110,7 +6112,7 @@ public final class RemoteProtocol {
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
private Builder(BuilderParent parent) {
|
||||||
super(parent);
|
super(parent);
|
||||||
maybeForceBuilderInitialization();
|
maybeForceBuilderInitialization();
|
||||||
}
|
}
|
||||||
|
|
@ -6522,11 +6524,10 @@ public final class RemoteProtocol {
|
||||||
"l\030\004 \001(\t\"\216\001\n\027DaemonMsgCreateProtocol\022\035\n\005p" +
|
"l\030\004 \001(\t\"\216\001\n\027DaemonMsgCreateProtocol\022\035\n\005p" +
|
||||||
"rops\030\001 \002(\0132\016.PropsProtocol\022\037\n\006deploy\030\002 \002" +
|
"rops\030\001 \002(\0132\016.PropsProtocol\022\037\n\006deploy\030\002 \002" +
|
||||||
"(\0132\017.DeployProtocol\022\014\n\004path\030\003 \002(\t\022%\n\nsup" +
|
"(\0132\017.DeployProtocol\022\014\n\004path\030\003 \002(\t\022%\n\nsup" +
|
||||||
"ervisor\030\004 \002(\0132\021.ActorRefProtocol\"\205\001\n\rPro",
|
"ervisor\030\004 \002(\0132\021.ActorRefProtocol\"^\n\rProp",
|
||||||
"psProtocol\022\022\n\ndispatcher\030\001 \002(\t\022\037\n\006deploy" +
|
"sProtocol\022\037\n\006deploy\030\002 \002(\0132\017.DeployProtoc" +
|
||||||
"\030\002 \002(\0132\017.DeployProtocol\022\030\n\020fromClassCrea" +
|
"ol\022\r\n\005clazz\030\003 \002(\t\022\014\n\004args\030\004 \003(\014\022\017\n\007class" +
|
||||||
"tor\030\003 \001(\t\022\017\n\007creator\030\004 \001(\014\022\024\n\014routerConf" +
|
"es\030\005 \003(\t\"g\n\016DeployProtocol\022\014\n\004path\030\001 \002(\t" +
|
||||||
"ig\030\005 \001(\014\"g\n\016DeployProtocol\022\014\n\004path\030\001 \002(\t" +
|
|
||||||
"\022\016\n\006config\030\002 \001(\014\022\024\n\014routerConfig\030\003 \001(\014\022\r" +
|
"\022\016\n\006config\030\002 \001(\014\022\024\n\014routerConfig\030\003 \001(\014\022\r" +
|
||||||
"\n\005scope\030\004 \001(\014\022\022\n\ndispatcher\030\005 \001(\t*7\n\013Com" +
|
"\n\005scope\030\004 \001(\014\022\022\n\ndispatcher\030\005 \001(\t*7\n\013Com" +
|
||||||
"mandType\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOWN\020\002\022\r\n\tH" +
|
"mandType\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOWN\020\002\022\r\n\tH" +
|
||||||
|
|
@ -6606,7 +6607,7 @@ public final class RemoteProtocol {
|
||||||
internal_static_PropsProtocol_fieldAccessorTable = new
|
internal_static_PropsProtocol_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_PropsProtocol_descriptor,
|
internal_static_PropsProtocol_descriptor,
|
||||||
new java.lang.String[] { "Dispatcher", "Deploy", "FromClassCreator", "Creator", "RouterConfig", },
|
new java.lang.String[] { "Deploy", "Clazz", "Args", "Classes", },
|
||||||
akka.remote.RemoteProtocol.PropsProtocol.class,
|
akka.remote.RemoteProtocol.PropsProtocol.class,
|
||||||
akka.remote.RemoteProtocol.PropsProtocol.Builder.class);
|
akka.remote.RemoteProtocol.PropsProtocol.Builder.class);
|
||||||
internal_static_DeployProtocol_descriptor =
|
internal_static_DeployProtocol_descriptor =
|
||||||
|
|
|
||||||
|
|
@ -95,11 +95,10 @@ message DaemonMsgCreateProtocol {
|
||||||
* Serialization of akka.actor.Props
|
* Serialization of akka.actor.Props
|
||||||
*/
|
*/
|
||||||
message PropsProtocol {
|
message PropsProtocol {
|
||||||
required string dispatcher = 1;
|
|
||||||
required DeployProtocol deploy = 2;
|
required DeployProtocol deploy = 2;
|
||||||
optional string fromClassCreator = 3;
|
required string clazz = 3;
|
||||||
optional bytes creator = 4;
|
repeated bytes args = 4;
|
||||||
optional bytes routerConfig = 5;
|
repeated string classes = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,6 @@ import akka.actor.{ Actor, ActorRef, Deploy, ExtendedActorSystem, NoScopeGiven,
|
||||||
import akka.remote.DaemonMsgCreate
|
import akka.remote.DaemonMsgCreate
|
||||||
import akka.remote.RemoteProtocol.{ DaemonMsgCreateProtocol, DeployProtocol, PropsProtocol }
|
import akka.remote.RemoteProtocol.{ DaemonMsgCreateProtocol, DeployProtocol, PropsProtocol }
|
||||||
import akka.routing.{ NoRouter, RouterConfig }
|
import akka.routing.{ NoRouter, RouterConfig }
|
||||||
import akka.actor.FromClassCreator
|
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import util.{ Failure, Success }
|
import util.{ Failure, Success }
|
||||||
|
|
||||||
|
|
@ -51,15 +50,11 @@ private[akka] class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) e
|
||||||
}
|
}
|
||||||
|
|
||||||
def propsProto = {
|
def propsProto = {
|
||||||
val builder = PropsProtocol.newBuilder.
|
val builder = PropsProtocol.newBuilder
|
||||||
setDispatcher(props.dispatcher).
|
.setClazz(props.clazz.getName)
|
||||||
setDeploy(deployProto(props.deploy))
|
.setDeploy(deployProto(props.deploy))
|
||||||
props.creator match {
|
props.args map serialize foreach builder.addArgs
|
||||||
case FromClassCreator(clazz) ⇒ builder.setFromClassCreator(clazz.getName)
|
props.args map (_.getClass.getName) foreach builder.addClasses
|
||||||
case creator ⇒ builder.setCreator(serialize(creator))
|
|
||||||
}
|
|
||||||
if (props.routerConfig != NoRouter)
|
|
||||||
builder.setRouterConfig(serialize(props.routerConfig))
|
|
||||||
builder.build
|
builder.build
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -95,21 +90,11 @@ private[akka] class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) e
|
||||||
}
|
}
|
||||||
|
|
||||||
def props = {
|
def props = {
|
||||||
val creator =
|
import scala.collection.JavaConverters._
|
||||||
if (proto.getProps.hasFromClassCreator)
|
val clazz = system.dynamicAccess.getClassFor[AnyRef](proto.getProps.getClazz).get
|
||||||
FromClassCreator(system.dynamicAccess.getClassFor[Actor](proto.getProps.getFromClassCreator).get)
|
val args: Vector[AnyRef] = (proto.getProps.getArgsList.asScala zip proto.getProps.getClassesList.asScala)
|
||||||
else
|
.map(p ⇒ deserialize(p._1, system.dynamicAccess.getClassFor[AnyRef](p._2).get))(collection.breakOut)
|
||||||
deserialize(proto.getProps.getCreator, classOf[() ⇒ Actor])
|
Props(deploy(proto.getProps.getDeploy), clazz, args)
|
||||||
|
|
||||||
val routerConfig =
|
|
||||||
if (proto.getProps.hasRouterConfig) deserialize(proto.getProps.getRouterConfig, classOf[RouterConfig])
|
|
||||||
else NoRouter
|
|
||||||
|
|
||||||
Props(
|
|
||||||
creator = creator,
|
|
||||||
dispatcher = proto.getProps.getDispatcher,
|
|
||||||
routerConfig = routerConfig,
|
|
||||||
deploy = deploy(proto.getProps.getDeploy))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
DaemonMsgCreate(
|
DaemonMsgCreate(
|
||||||
|
|
@ -119,7 +104,7 @@ private[akka] class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) e
|
||||||
supervisor = deserializeActorRef(system, proto.getSupervisor))
|
supervisor = deserializeActorRef(system, proto.getSupervisor))
|
||||||
}
|
}
|
||||||
|
|
||||||
protected def serialize(any: AnyRef): ByteString = ByteString.copyFrom(serialization.serialize(any).get)
|
protected def serialize(any: Any): ByteString = ByteString.copyFrom(serialization.serialize(any.asInstanceOf[AnyRef]).get)
|
||||||
|
|
||||||
protected def deserialize[T: ClassTag](data: ByteString, clazz: Class[T]): T = {
|
protected def deserialize[T: ClassTag](data: ByteString, clazz: Class[T]): T = {
|
||||||
val bytes = data.toByteArray
|
val bytes = data.toByteArray
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import language.postfixOps
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.actor.{ Actor, Address, Props, Deploy, OneForOneStrategy, SupervisorStrategy, FromClassCreator }
|
import akka.actor.{ Actor, Address, Props, Deploy, OneForOneStrategy, SupervisorStrategy }
|
||||||
import akka.remote.{ DaemonMsgCreate, RemoteScope }
|
import akka.remote.{ DaemonMsgCreate, RemoteScope }
|
||||||
import akka.routing.{ RoundRobinRouter, FromConfig }
|
import akka.routing.{ RoundRobinRouter, FromConfig }
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
@ -87,11 +87,13 @@ class DaemonMsgCreateSerializerSpec extends AkkaSpec {
|
||||||
|
|
||||||
def assertDaemonMsgCreate(expected: DaemonMsgCreate, got: DaemonMsgCreate): Unit = {
|
def assertDaemonMsgCreate(expected: DaemonMsgCreate, got: DaemonMsgCreate): Unit = {
|
||||||
// can't compare props.creator when function
|
// can't compare props.creator when function
|
||||||
if (expected.props.creator.isInstanceOf[FromClassCreator])
|
assert(got.props.clazz === expected.props.clazz)
|
||||||
assert(got.props.creator === expected.props.creator)
|
assert(got.props.args.length === expected.props.args.length)
|
||||||
assert(got.props.dispatcher === expected.props.dispatcher)
|
got.props.args zip expected.props.args foreach {
|
||||||
assert(got.props.dispatcher === expected.props.dispatcher)
|
case (g, e) ⇒
|
||||||
assert(got.props.routerConfig === expected.props.routerConfig)
|
if (e.isInstanceOf[Function0[_]]) ()
|
||||||
|
else assert(g === e)
|
||||||
|
}
|
||||||
assert(got.props.deploy === expected.props.deploy)
|
assert(got.props.deploy === expected.props.deploy)
|
||||||
assert(got.deploy === expected.deploy)
|
assert(got.deploy === expected.deploy)
|
||||||
assert(got.path === expected.path)
|
assert(got.path === expected.path)
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue