Deprecate actorFor in favor of ActorSelection, see #3074

* Deprecate all actorFor methods
* resolveActorRef in provider
* Identify auto receive message
* Support ActorPath in actorSelection
* Support remote actor selections
* Additional tests of actor selection
* Update tests (keep most actorFor tests)
* Update samples to use actorSelection
* Updates to documentation
* Migration guide, including motivation
This commit is contained in:
Patrik Nordwall 2013-03-26 18:17:50 +01:00
parent 641e499cc4
commit 887af975ae
80 changed files with 1496 additions and 619 deletions

View file

@ -9,7 +9,6 @@ import akka.testkit._
import scala.concurrent.duration._
import scala.concurrent.Await
import akka.pattern.ask
import java.net.MalformedURLException
object ActorLookupSpec {
@ -285,56 +284,4 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
}
"An ActorSelection" must {
"send messages directly" in {
ActorSelection(c1, "") ! GetSender(testActor)
expectMsg(system.deadLetters)
lastSender must be === c1
}
"send messages with correct sender" in {
implicit val sender = c1
ActorSelection(c21, "../../*") ! GetSender(testActor)
val actors = Set() ++ receiveWhile(messages = 2) {
case `c1` lastSender
}
actors must be === Set(c1, c2)
expectNoMsg(1 second)
}
"drop messages which cannot be delivered" in {
implicit val sender = c2
ActorSelection(c21, "../../*/c21") ! GetSender(testActor)
val actors = receiveWhile(messages = 2) {
case `c2` lastSender
}
actors must be === Seq(c21)
expectNoMsg(1 second)
}
}
"An ActorPath" must {
"support parsing its String rep" in {
val path = system.actorFor("user").path
ActorPath.fromString(path.toString) must be(path)
}
"support parsing remote paths" in {
val remote = "akka://sys@host:1234/some/ref"
ActorPath.fromString(remote).toString must be(remote)
}
"throw exception upon malformed paths" in {
intercept[MalformedURLException] { ActorPath.fromString("") }
intercept[MalformedURLException] { ActorPath.fromString("://hallo") }
intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") }
intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") }
intercept[MalformedURLException] { ActorPath.fromString("a://l:1/b") }
}
}
}

View file

@ -5,10 +5,29 @@ package akka.actor
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import java.net.MalformedURLException
class ActorPathSpec extends WordSpec with MustMatchers {
"ActorPath" must {
"An ActorPath" must {
"support parsing its String rep" in {
val path = RootActorPath(Address("akka.tcp", "mysys")) / "user"
ActorPath.fromString(path.toString) must be(path)
}
"support parsing remote paths" in {
val remote = "akka://sys@host:1234/some/ref"
ActorPath.fromString(remote).toString must be(remote)
}
"throw exception upon malformed paths" in {
intercept[MalformedURLException] { ActorPath.fromString("") }
intercept[MalformedURLException] { ActorPath.fromString("://hallo") }
intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") }
intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") }
intercept[MalformedURLException] { ActorPath.fromString("a://l:1/b") }
}
"create correct toString" in {
val a = Address("akka.tcp", "mysys")
@ -18,6 +37,10 @@ class ActorPathSpec extends WordSpec with MustMatchers {
(RootActorPath(a) / "user" / "foo" / "bar").toString must be("akka.tcp://mysys/user/foo/bar")
}
"have correct path elements" in {
(RootActorPath(Address("akka.tcp", "mysys")) / "user" / "foo" / "bar").elements.toSeq must be(Seq("user", "foo", "bar"))
}
"create correct toStringWithAddress" in {
val local = Address("akka.tcp", "mysys")
val a = local.copy(host = Some("aaa"), port = Some(2552))

View file

@ -0,0 +1,292 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import language.postfixOps
import akka.testkit._
import scala.concurrent.duration._
import scala.concurrent.Await
import akka.pattern.ask
object ActorSelectionSpec {
case class Create(child: String)
trait Query
case class SelectString(path: String) extends Query
case class SelectPath(path: ActorPath) extends Query
case class GetSender(to: ActorRef) extends Query
val p = Props[Node]
class Node extends Actor {
def receive = {
case Create(name) sender ! context.actorOf(p, name)
case SelectString(path) sender ! context.actorSelection(path)
case SelectPath(path) sender ! context.actorSelection(path)
case GetSender(ref) ref ! sender
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorSelectionSpec extends AkkaSpec("akka.loglevel=DEBUG") with DefaultTimeout {
import ActorSelectionSpec._
val c1 = system.actorOf(p, "c1")
val c2 = system.actorOf(p, "c2")
val c21 = Await.result((c2 ? Create("c21")).mapTo[ActorRef], timeout.duration)
val sysImpl = system.asInstanceOf[ActorSystemImpl]
val user = sysImpl.guardian
val syst = sysImpl.systemGuardian
val root = sysImpl.lookupRoot
def empty(path: String) =
new EmptyLocalActorRef(sysImpl.provider, path match {
case RelativeActorPath(elems) sysImpl.lookupRoot.path / elems
}, system.eventStream)
val idProbe = TestProbe()
def identify(selection: ActorSelection): Option[ActorRef] = {
selection.tell(Identify(selection), idProbe.ref)
idProbe.expectMsgPF() {
case ActorIdentity(`selection`, ref) ref
}
}
def identify(path: String): Option[ActorRef] = identify(system.actorSelection(path))
def identify(path: ActorPath): Option[ActorRef] = identify(system.actorSelection(path))
def askNode(node: ActorRef, query: Query): Option[ActorRef] = {
Await.result(node ? query, timeout.duration) match {
case ref: ActorRef Some(ref)
case selection: ActorSelection identify(selection)
}
}
"An ActorSystem" must {
"select actors by their path" in {
identify(c1.path) must be === Some(c1)
identify(c2.path) must be === Some(c2)
identify(c21.path) must be === Some(c21)
identify(system / "c1") must be === Some(c1)
identify(system / "c2") must be === Some(c2)
identify(system / "c2" / "c21") must be === Some(c21)
identify(system child "c2" child "c21") must be === Some(c21) // test Java API
identify(system / Seq("c2", "c21")) must be === Some(c21)
import scala.collection.JavaConverters._
identify(system descendant Seq("c2", "c21").asJava) // test Java API
}
"select actors by their string path representation" in {
identify(c1.path.toString) must be === Some(c1)
identify(c2.path.toString) must be === Some(c2)
identify(c21.path.toString) must be === Some(c21)
identify(c1.path.elements.mkString("/", "/", "")) must be === Some(c1)
identify(c2.path.elements.mkString("/", "/", "")) must be === Some(c2)
identify(c21.path.elements.mkString("/", "/", "")) must be === Some(c21)
}
"take actor incarnation into account when comparing actor references" in {
val name = "abcdefg"
val a1 = system.actorOf(p, name)
watch(a1)
a1 ! PoisonPill
expectMsgType[Terminated].actor must be === a1
// not equal because it's terminated
identify(a1.path) must be === None
val a2 = system.actorOf(p, name)
a2.path must be(a1.path)
a2.path.toString must be(a1.path.toString)
a2 must not be (a1)
a2.toString must not be (a1.toString)
watch(a2)
a2 ! PoisonPill
expectMsgType[Terminated].actor must be === a2
}
"select actors by their root-anchored relative path" in {
identify(c1.path.elements.mkString("/", "/", "")) must be === Some(c1)
identify(c2.path.elements.mkString("/", "/", "")) must be === Some(c2)
identify(c21.path.elements.mkString("/", "/", "")) must be === Some(c21)
}
"select actors by their relative path" in {
identify(c1.path.elements.mkString("/")) must be === Some(c1)
identify(c2.path.elements.mkString("/")) must be === Some(c2)
identify(c21.path.elements.mkString("/")) must be === Some(c21)
}
"select system-generated actors" in {
identify("/user") must be === Some(user)
identify("/deadLetters") must be === Some(system.deadLetters)
identify("/system") must be === Some(syst)
identify(syst.path) must be === Some(syst)
identify(syst.path.elements.mkString("/", "/", "")) must be === Some(syst)
identify("/") must be === Some(root)
identify("") must be === Some(root)
identify(RootActorPath(root.path.address)) must be === Some(root)
identify("..") must be === Some(root)
identify(root.path) must be === Some(root)
identify(root.path.elements.mkString("/", "/", "")) must be === Some(root)
identify("user") must be === Some(user)
identify("deadLetters") must be === Some(system.deadLetters)
identify("system") must be === Some(syst)
identify("user/") must be === Some(user)
identify("deadLetters/") must be === Some(system.deadLetters)
identify("system/") must be === Some(syst)
}
"return deadLetters or ActorIdentity(None), respectively, for non-existing paths" in {
identify("a/b/c") must be === None
identify("a/b/c") must be === None
identify("akka://all-systems/Nobody") must be === None
identify("akka://all-systems/user") must be === None
identify(system / "hallo") must be === None
}
}
"An ActorContext" must {
val all = Seq(c1, c2, c21)
"select actors by their path" in {
def check(looker: ActorRef, pathOf: ActorRef, result: ActorRef) {
askNode(looker, SelectPath(pathOf.path)) must be === Some(result)
}
for {
looker all
target all
} check(looker, target, target)
}
"select actors by their string path representation" in {
def check(looker: ActorRef, pathOf: ActorRef, result: ActorRef) {
askNode(looker, SelectString(pathOf.path.elements.mkString("/", "/", ""))) must be === Some(result)
// with trailing /
askNode(looker, SelectString(pathOf.path.elements.mkString("/", "/", "") + "/")) must be === Some(result)
}
for {
looker all
target all
} check(looker, target, target)
}
"select actors by their root-anchored relative path" in {
def check(looker: ActorRef, pathOf: ActorRef, result: ActorRef) {
askNode(looker, SelectString(pathOf.path.elements.mkString("/", "/", ""))) must be === Some(result)
askNode(looker, SelectString(pathOf.path.elements.mkString("/", "/", "/"))) must be === Some(result)
}
for {
looker all
target all
} check(looker, target, target)
}
"select actors by their relative path" in {
def check(looker: ActorRef, result: ActorRef, elems: String*) {
askNode(looker, SelectString(elems mkString "/")) must be === Some(result)
askNode(looker, SelectString(elems mkString ("", "/", "/"))) must be === Some(result)
}
check(c1, user, "..")
for {
looker Seq(c1, c2)
target all
} check(looker, target, Seq("..") ++ target.path.elements.drop(1): _*)
check(c21, user, "..", "..")
check(c21, root, "..", "..", "..")
check(c21, root, "..", "..", "..", "..")
}
"find system-generated actors" in {
def check(target: ActorRef) {
for (looker all) {
askNode(looker, SelectPath(target.path)) must be === Some(target)
askNode(looker, SelectString(target.path.toString)) must be === Some(target)
askNode(looker, SelectString(target.path.toString + "/")) must be === Some(target)
}
if (target != root)
askNode(c1, SelectString("../.." + target.path.elements.mkString("/", "/", "/"))) must be === Some(target)
}
for (target Seq(root, syst, user)) check(target)
}
"return deadLetters or ActorIdentity(None), respectively, for non-existing paths" in {
import scala.collection.JavaConverters._
def checkOne(looker: ActorRef, query: Query, result: Option[ActorRef]) {
val lookup = askNode(looker, query)
lookup must be === result
}
def check(looker: ActorRef) {
val lookname = looker.path.elements.mkString("", "/", "/")
for (
(l, r) Seq(
SelectString("a/b/c") -> None,
SelectString("akka://all-systems/Nobody") -> None,
SelectPath(system / "hallo") -> None,
SelectPath(looker.path child "hallo") -> None, // test Java API
SelectPath(looker.path descendant Seq("a", "b").asJava) -> None) // test Java API
) checkOne(looker, l, r)
}
for (looker all) check(looker)
}
}
"An ActorSelection" must {
"send messages directly" in {
ActorSelection(c1, "") ! GetSender(testActor)
expectMsg(system.deadLetters)
lastSender must be === c1
}
"send messages to string path" in {
system.actorSelection("/user/c2/c21") ! GetSender(testActor)
expectMsg(system.deadLetters)
lastSender must be === c21
}
"send messages to actor path" in {
system.actorSelection(system / "c2" / "c21") ! GetSender(testActor)
expectMsg(system.deadLetters)
lastSender must be === c21
}
"send messages with correct sender" in {
implicit val sender = c1
ActorSelection(c21, "../../*") ! GetSender(testActor)
val actors = Set() ++ receiveWhile(messages = 2) {
case `c1` lastSender
}
actors must be === Set(c1, c2)
expectNoMsg(1 second)
}
"drop messages which cannot be delivered" in {
implicit val sender = c2
ActorSelection(c21, "../../*/c21") ! GetSender(testActor)
val actors = receiveWhile(messages = 2) {
case `c2` lastSender
}
actors must be === Seq(c21)
expectNoMsg(1 second)
}
}
}

View file

@ -252,7 +252,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
"shut down when /user fails" in {
implicit val system = ActorSystem("Stop", AkkaSpec.testConf)
EventFilter[ActorKilledException]() intercept {
system.actorFor("/user") ! Kill
system.actorSelection("/user") ! Kill
awaitCond(system.isTerminated)
}
}

View file

@ -5,10 +5,11 @@
package akka.actor
import akka.AkkaException
import scala.collection.immutable
import scala.annotation.tailrec
import scala.reflect.BeanProperty
import scala.util.control.NoStackTrace
import java.util.regex.Pattern
import scala.annotation.tailrec
/**
* INTERNAL API
@ -54,6 +55,30 @@ case object Kill extends Kill {
def getInstance = this
}
/**
* A message all Actors will understand, that when processed will reply with
* [[akka.actor.ActorIdentity]] containing the `ActorRef`. The `messageId`
* is returned in the `ActorIdentity` message as `correlationId`.
*/
@SerialVersionUID(1L)
case class Identify(messageId: Any) extends AutoReceivedMessage
/**
* Reply to [[akka.actor.Identify]]. Contains
* `Some(ref)` with the `ActorRef` of the actor replying to the request or
* `None` if no actor matched the request.
* The `correlationId` is taken from the `messageId` in
* the `Identify` message.
*/
@SerialVersionUID(1L)
case class ActorIdentity(correlationId: Any, ref: Option[ActorRef]) {
/**
* Java API: `ActorRef` of the actor replying to the request or
* null if no actor matched the request.
*/
def getRef: ActorRef = ref.orNull
}
/**
* When Death Watch is used, the watcher will receive a Terminated(watched)
* message when watched is terminated.
@ -100,17 +125,45 @@ case object ReceiveTimeout extends ReceiveTimeout {
}
/**
* INTERNAL API
* ActorRefFactory.actorSelection returns a special ref which sends these
* nested path descriptions whenever using ! on them, the idea being that the
* message is delivered by active routing of the various actors involved.
*/
sealed trait SelectionPath extends AutoReceivedMessage with PossiblyHarmful
private[akka] sealed trait SelectionPath extends AutoReceivedMessage with PossiblyHarmful
/**
* INTERNAL API
*/
@SerialVersionUID(1L)
private[akka] case class SelectChildName(name: String, next: Any) extends SelectionPath
private[akka] case class SelectChildName(name: String, next: Any) extends SelectionPath {
def wrappedMessage: Any = {
@tailrec def rec(nx: Any): Any = nx match {
case SelectChildName(_, n) rec(n)
case SelectChildPattern(_, n) rec(n)
case SelectParent(n) rec(n)
case x x
}
rec(next)
}
def identifyRequest: Option[Identify] = wrappedMessage match {
case x: Identify Some(x)
case _ None
}
def allChildNames: immutable.Iterable[String] = {
@tailrec def rec(nx: Any, acc: List[String]): List[String] = nx match {
case SelectChildName(name, n) rec(n, name :: acc)
case SelectChildPattern(_, n) throw new IllegalArgumentException("SelectChildPattern not allowed")
case SelectParent(n) if acc.isEmpty rec(n, acc)
case SelectParent(n) rec(n, acc.tail)
case _ acc
}
rec(this, Nil).reverse
}
}
/**
* INTERNAL API

View file

@ -103,12 +103,12 @@ trait ActorContext extends ActorRefFactory {
/**
* Returns all supervised children; this method returns a view (i.e. a lazy
* collection) onto the internal collection of children. Targeted lookups
* should be using `actorFor` instead for performance reasons:
* should be using `child` instead for performance reasons:
*
* {{{
* val badLookup = context.children find (_.path.name == "kid")
* // should better be expressed as:
* val goodLookup = context.actorFor("kid")
* val goodLookup = context.child("kid")
* }}}
*/
def children: immutable.Iterable[ActorRef]
@ -475,9 +475,27 @@ private[akka] class ActorCell(
case AddressTerminated(address) addressTerminated(address)
case Kill throw new ActorKilledException("Kill")
case PoisonPill self.stop()
case SelectParent(m) parent.tell(m, msg.sender)
case SelectChildName(name, m) getChildByName(name) match { case Some(c: ChildRestartStats) c.child.tell(m, msg.sender); case _ }
case SelectParent(m)
if (self == system.provider.rootGuardian) self.tell(m, msg.sender)
else parent.tell(m, msg.sender)
case s @ SelectChildName(name, m)
def selectChild(): Unit = {
getChildByName(name) match {
case Some(c: ChildRestartStats) c.child.tell(m, msg.sender)
case _
s.identifyRequest foreach { x sender ! ActorIdentity(x.messageId, None) }
}
}
// need this special case because of extraNames handled by rootGuardian
if (self == system.provider.rootGuardian) {
self.asInstanceOf[LocalActorRef].getSingleChild(name) match {
case Nobody selectChild()
case child child.tell(m, msg.sender)
}
} else
selectChild()
case SelectChildPattern(p, m) for (c children if p.matcher(c.path.name).matches) c.tell(m, msg.sender)
case Identify(messageId) sender ! ActorIdentity(messageId, Some(self))
}
}

View file

@ -180,7 +180,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
*/
override private[akka] def withUid(uid: Int): ActorPath =
if (uid == ActorCell.undefinedUid) this
else throw new IllegalStateException("RootActorPath must not have uid")
else throw new IllegalStateException(s"RootActorPath must have undefinedUid, [$uid != ${ActorCell.undefinedUid}")
}

View file

@ -116,7 +116,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
* there is nobody to reply to).
*
* <pre>
* actor.tell(message, context);
* actor.tell(message, getSelf());
* </pre>
*/
final def tell(msg: Any, sender: ActorRef): Unit = this.!(msg)(sender)
@ -335,8 +335,9 @@ private[akka] class LocalActorRef private[akka] (
/**
* Method for looking up a single child beneath this actor. Override in order
* to inject synthetic actor paths like /temp.
* It is racy if called from the outside.
*/
protected def getSingleChild(name: String): InternalActorRef = {
def getSingleChild(name: String): InternalActorRef = {
val (childName, uid) = ActorCell.splitNameAndUid(name)
actorCell.getChildByName(childName) match {
case Some(crs: ChildRestartStats) if uid == ActorCell.undefinedUid || uid == crs.uid
@ -401,7 +402,7 @@ private[akka] case class SerializedActorRef private (path: String) {
"Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use 'akka.serialization.Serialization.currentSystem.withValue(system) { ... }'")
case someSystem
someSystem.actorFor(path)
someSystem.provider.resolveActorRef(path)
}
}
@ -473,24 +474,30 @@ private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
override def sendSystemMessage(message: SystemMessage): Unit = {
if (Mailbox.debug) println(s"ELAR $path having enqueued $message")
specialHandle(message)
specialHandle(message, provider.deadLetters)
}
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = message match {
case null throw new InvalidMessageException("Message is null")
case d: DeadLetter
specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend!
case _ if !specialHandle(message)
specialHandle(d.message, d.sender) // do NOT form endless loops, since deadLetters will resend!
case _ if !specialHandle(message, sender)
eventStream.publish(DeadLetter(message, if (sender eq Actor.noSender) provider.deadLetters else sender, this))
case _
}
protected def specialHandle(msg: Any): Boolean = msg match {
protected def specialHandle(msg: Any, sender: ActorRef): Boolean = msg match {
case w: Watch
if (w.watchee == this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false, addressTerminated = false)
true
case _: Unwatch true // Just ignore
case Identify(messageId)
sender ! ActorIdentity(messageId, None)
true
case s: SelectChildName
s.identifyRequest foreach { x sender ! ActorIdentity(x.messageId, None) }
true
case _ false
}
}
@ -507,17 +514,24 @@ private[akka] class DeadLetterActorRef(_provider: ActorRefProvider,
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
case null throw new InvalidMessageException("Message is null")
case d: DeadLetter if (!specialHandle(d.message)) eventStream.publish(d)
case _ if (!specialHandle(message))
case Identify(messageId) sender ! ActorIdentity(messageId, Some(this))
case d: DeadLetter if (!specialHandle(d.message, d.sender)) eventStream.publish(d)
case _ if (!specialHandle(message, sender))
eventStream.publish(DeadLetter(message, if (sender eq Actor.noSender) provider.deadLetters else sender, this))
}
override protected def specialHandle(msg: Any): Boolean = msg match {
override protected def specialHandle(msg: Any, sender: ActorRef): Boolean = msg match {
case w: Watch
if (w.watchee != this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false, addressTerminated = false)
true
case w: Unwatch true // Just ignore
case Identify(messageId)
sender ! ActorIdentity(messageId, None)
true
case s: SelectChildName
s.identifyRequest foreach { x sender ! ActorIdentity(x.messageId, None) }
true
case NullMessage true
case _ false
}

View file

@ -4,6 +4,7 @@
package akka.actor
import scala.collection.immutable
import akka.dispatch.sysmsg._
import akka.dispatch.NullMessage
import akka.routing._
@ -28,6 +29,13 @@ trait ActorRefProvider {
*/
def rootGuardian: InternalActorRef
/**
* Reference to the supervisor of guardian and systemGuardian at the specified address;
* this is exposed so that the ActorRefFactory can use it as lookupRoot, i.e.
* for anchoring absolute actor selections.
*/
def rootGuardianAt(address: Address): ActorRef
/**
* Reference to the supervisor used for all top-level user actors.
*/
@ -109,6 +117,7 @@ trait ActorRefProvider {
* Create actor reference for a specified local or remote path. If no such
* actor exists, it will be (equivalent to) a dead letter reference.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: ActorPath): InternalActorRef
/**
@ -117,6 +126,7 @@ trait ActorRefProvider {
* (equivalent to) a dead letter reference. If `s` is a relative URI, resolve
* it relative to the given ref.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, s: String): InternalActorRef
/**
@ -125,8 +135,21 @@ trait ActorRefProvider {
* i.e. it cannot be used to obtain a reference to an actor which is not
* physically or logically attached to this actor system.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, p: Iterable[String]): InternalActorRef
/**
* Create actor reference for a specified path. If no such
* actor exists, it will be (equivalent to) a dead letter reference.
*/
def resolveActorRef(path: String): ActorRef
/**
* Create actor reference for a specified path. If no such
* actor exists, it will be (equivalent to) a dead letter reference.
*/
def resolveActorRef(path: ActorPath): ActorRef
/**
* This Future is completed upon termination of this ActorRefProvider, which
* is usually initiated by stopping the guardian via ActorSystem.stop().
@ -214,6 +237,7 @@ trait ActorRefFactory {
* `watch(ref)` to be notified of the targets termination, which is also
* signaled if the queried path cannot be resolved.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: ActorPath): ActorRef = provider.actorFor(path)
/**
@ -230,6 +254,7 @@ trait ActorRefFactory {
* relative to the current context as described for look-ups by
* `actorOf(Iterable[String])`
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: String): ActorRef = provider.actorFor(lookupRoot, path)
/**
@ -250,6 +275,7 @@ trait ActorRefFactory {
*
* For maximum performance use a collection with efficient head & tail operations.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: Iterable[String]): ActorRef = provider.actorFor(lookupRoot, path)
/**
@ -273,6 +299,7 @@ trait ActorRefFactory {
*
* For maximum performance use a collection with efficient head & tail operations.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: java.lang.Iterable[String]): ActorRef = provider.actorFor(lookupRoot, immutableSeq(path))
/**
@ -282,7 +309,26 @@ trait ActorRefFactory {
* the supplied path, it is recommended to send a message and gather the
* replies in order to resolve the matching set of actors.
*/
def actorSelection(path: String): ActorSelection = ActorSelection(lookupRoot, path)
def actorSelection(path: String): ActorSelection = path match {
case RelativeActorPath(elems)
if (elems.isEmpty) ActorSelection(provider.deadLetters, "")
else if (elems.head.isEmpty) ActorSelection(provider.rootGuardian, elems.tail)
else ActorSelection(lookupRoot, elems)
case ActorPathExtractor(address, elems)
ActorSelection(provider.rootGuardianAt(address), elems)
case _
ActorSelection(provider.deadLetters, "")
}
/**
* Construct an [[akka.actor.ActorSelection]] from the given path, which is
* parsed for wildcards (these are replaced by regular expressions
* internally). No attempt is made to verify the existence of any part of
* the supplied path, it is recommended to send a message and gather the
* replies in order to resolve the matching set of actors.
*/
def actorSelection(path: ActorPath): ActorSelection =
ActorSelection(provider.rootGuardianAt(path.address), path.elements)
/**
* Stop the actor pointed to by the given [[akka.actor.ActorRef]]; this is
@ -505,7 +551,7 @@ private[akka] class LocalActorRefProvider private[akka] (
*/
protected def systemGuardianStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy
lazy val rootGuardian: LocalActorRef =
override lazy val rootGuardian: LocalActorRef =
new LocalActorRef(system, Props(new Guardian(rootGuardianStrategy)), theOneWhoWalksTheBubblesOfSpaceTime, rootPath) {
override def getParent: InternalActorRef = this
override def getSingleChild(name: String): InternalActorRef = name match {
@ -515,7 +561,11 @@ private[akka] class LocalActorRefProvider private[akka] (
}
}
lazy val guardian: LocalActorRef = {
override def rootGuardianAt(address: Address): ActorRef =
if (address == rootPath.address) rootGuardian
else deadLetters
override lazy val guardian: LocalActorRef = {
val cell = rootGuardian.underlying
cell.reserveChild("user")
val ref = new LocalActorRef(system, Props(new Guardian(guardianStrategy)), rootGuardian, rootPath / "user")
@ -524,7 +574,7 @@ private[akka] class LocalActorRefProvider private[akka] (
ref
}
lazy val systemGuardian: LocalActorRef = {
override lazy val systemGuardian: LocalActorRef = {
val cell = rootGuardian.underlying
cell.reserveChild("system")
val ref = new LocalActorRef(system, Props(new SystemGuardian(systemGuardianStrategy)), rootGuardian, rootPath / "system")
@ -554,7 +604,8 @@ private[akka] class LocalActorRefProvider private[akka] (
eventStream.startDefaultLoggers(_system)
}
def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
@deprecated("use actorSelection instead of actorFor", "2.2")
override def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
case RelativeActorPath(elems)
if (elems.isEmpty) {
log.debug("look-up of empty path string [{}] fails (per definition)", path)
@ -567,14 +618,16 @@ private[akka] class LocalActorRefProvider private[akka] (
deadLetters
}
def actorFor(path: ActorPath): InternalActorRef =
@deprecated("use actorSelection instead of actorFor", "2.2")
override def actorFor(path: ActorPath): InternalActorRef =
if (path.root == rootPath) actorFor(rootGuardian, path.elements)
else {
log.debug("look-up of foreign ActorPath [{}] failed", path)
deadLetters
}
def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef =
@deprecated("use actorSelection instead of actorFor", "2.2")
override def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef =
if (path.isEmpty) {
log.debug("look-up of empty path sequence fails (per definition)")
deadLetters
@ -585,6 +638,35 @@ private[akka] class LocalActorRefProvider private[akka] (
case x x
}
def resolveActorRef(path: String): ActorRef = path match {
case ActorPathExtractor(address, elems) if address == rootPath.address resolveActorRef(rootGuardian, elems)
case _
log.debug("resolve of unknown path [{}] failed", path)
deadLetters
}
def resolveActorRef(path: ActorPath): ActorRef = {
if (path.root == rootPath) resolveActorRef(rootGuardian, path.elements)
else {
log.debug("resolve of foreign ActorPath [{}] failed", path)
deadLetters
}
}
/**
* INTERNAL API
*/
private[akka] def resolveActorRef(ref: InternalActorRef, pathElements: Iterable[String]): InternalActorRef =
if (pathElements.isEmpty) {
log.debug("resolve of empty path sequence fails (per definition)")
deadLetters
} else ref.getChild(pathElements.iterator) match {
case Nobody
log.debug("resolve of path sequence [/{}] failed", pathElements.mkString("/"))
new EmptyLocalActorRef(system.provider, ref.path / pathElements, eventStream)
case x x
}
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
props.routerConfig match {

View file

@ -4,7 +4,7 @@
package akka.actor
import language.implicitConversions
import scala.collection.immutable
import java.util.regex.Pattern
import akka.util.Helpers
@ -12,13 +12,15 @@ import akka.util.Helpers
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
* allowing for broadcasting of messages to that section.
*/
abstract class ActorSelection {
@SerialVersionUID(1L)
abstract class ActorSelection extends Serializable {
this: ScalaActorSelection
protected def target: ActorRef
protected def path: Array[AnyRef]
@deprecated("use the two-arg variant (typically getSelf() as second arg)", "2.2")
def tell(msg: Any): Unit = target ! toMessage(msg, path)
def tell(msg: Any, sender: ActorRef): Unit = target.tell(toMessage(msg, path), sender)
@ -36,6 +38,23 @@ abstract class ActorSelection {
}
acc
}
override def toString: String = {
val sb = new java.lang.StringBuilder
sb.append("ActorSelection[").
append(target.toString).
append(path.mkString("/", "/", "")).
append("]")
sb.toString
}
override def equals(obj: Any): Boolean = obj match {
case s: ActorSelection this.target == s.target && this.path == s.path
case _ false
}
override def hashCode: Int =
37 * (37 * 17 + target.hashCode) + path.hashCode
}
/**
@ -60,6 +79,23 @@ object ActorSelection {
def path = compiled
}
}
/**
* Construct an ActorSelection from the given string representing a path
* relative to the given target. This operation has to create all the
* matching magic, so it is preferable to cache its result if the
* intention is to send messages frequently.
*/
def apply(anchor: ActorRef, elements: immutable.Iterable[String]): ActorSelection = {
// TODO #3073 optimize/align compiled Array
val elems: Array[String] = elements.collect { case x if x.nonEmpty x }(collection.breakOut)
val compiled: Array[AnyRef] = elems map (x if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) Helpers.makePattern(x) else x)
new ActorSelection with ScalaActorSelection {
override def target = anchor
override def path = compiled
}
}
}
/**

View file

@ -259,7 +259,7 @@ abstract class ActorSystem extends ActorRefFactory {
def logConfiguration(): Unit
/**
* Construct a path below the application guardian to be used with [[ActorSystem.actorFor]].
* Construct a path below the application guardian to be used with [[ActorSystem.actorSelection]].
*/
def /(name: String): ActorPath
@ -269,7 +269,7 @@ abstract class ActorSystem extends ActorRefFactory {
def child(child: String): ActorPath = /(child)
/**
* Construct a path below the application guardian to be used with [[ActorSystem.actorFor]].
* Construct a path below the application guardian to be used with [[ActorSystem.actorSelection]].
*/
def /(name: Iterable[String]): ActorPath
@ -725,7 +725,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
indent + node.path.name + " " + Logging.simpleName(node)
}
}
printNode(actorFor("/"), "")
printNode(lookupRoot, "")
}
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {

View file

@ -258,6 +258,9 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
*/
def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(immutableSeq(routees))
// TODO replace all usages of actorFor in routing with actorSelection. Not possible until
// major refactoring of routing because of demand for synchronous registration of routees.
/**
* Looks up routes with specified paths and registers them.
*/

View file

@ -162,6 +162,8 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
case Failure(e: TimeoutException) exchange.setFailure(FailureResult(new TimeoutException("Failed to get Ack or Failure response from the actor [%s] within timeout [%s]. Check replyTimeout and blocking settings [%s]" format (endpoint.path, endpoint.replyTimeout, endpoint))))
case Failure(throwable) exchange.setFailure(FailureResult(throwable))
}
// FIXME #3074 how do we solve this with actorSelection?
val async = try actorFor(endpoint.path).ask(messageFor(exchange))(Timeout(endpoint.replyTimeout)) catch { case NonFatal(e) Future.failed(e) }
implicit val ec = camel.system.dispatcher // FIXME which ExecutionContext should be used here?
async.onComplete(action andThen { _ callback.done(false) })
@ -170,6 +172,7 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
}
// FIXME #3074 how do we solve this with actorSelection?
private def fireAndForget(message: CamelMessage, exchange: CamelExchangeAdapter): Unit =
try { actorFor(endpoint.path) ! message } catch { case NonFatal(e) exchange.setFailure(new FailureResult(e)) }
@ -205,6 +208,7 @@ private[camel] case class ActorEndpointPath private (actorPath: String) {
require(actorPath.startsWith("akka://"))
def findActorIn(system: ActorSystem): Option[ActorRef] = {
// FIXME #3074 how do we solve this with actorSelection?
val ref = system.actorFor(actorPath)
if (ref.isTerminated) None else Some(ref)
}

View file

@ -70,13 +70,21 @@ private[akka] class ClusterActorRefProvider(
* This method is overridden here to keep track of remote deployed actors to
* be able to clean up corresponding child references.
*/
override def useActorOnNode(path: ActorPath, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
super.useActorOnNode(path, props, deploy, supervisor)
remoteDeploymentWatcher ! ((actorFor(path), supervisor))
override def useActorOnNode(ref: ActorRef, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
super.useActorOnNode(ref, props, deploy, supervisor)
import RemoteDeploymentWatcher.WatchRemote
remoteDeploymentWatcher ! WatchRemote(ref, supervisor)
}
}
/**
* INTERNAL API
*/
private[akka] object RemoteDeploymentWatcher {
case class WatchRemote(actor: ActorRef, supervisor: ActorRef)
}
/**
* INTERNAL API
*
@ -84,10 +92,11 @@ private[akka] class ClusterActorRefProvider(
* goes down (jvm crash, network failure), i.e. triggered by [[akka.actor.AddressTerminated]].
*/
private[akka] class RemoteDeploymentWatcher extends Actor {
import RemoteDeploymentWatcher._
var supervisors = Map.empty[ActorRef, InternalActorRef]
def receive = {
case (a: ActorRef, supervisor: InternalActorRef)
case WatchRemote(a, supervisor: InternalActorRef)
supervisors += (a -> supervisor)
context.watch(a)

View file

@ -20,6 +20,7 @@ import akka.pattern.ask
import akka.util.Timeout
import akka.cluster.MemberStatus._
import akka.cluster.ClusterEvent._
import akka.actor.ActorSelection
/**
* Base trait for all cluster messages. All ClusterMessage's are serializable.
@ -244,8 +245,8 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
/**
* Looks up and returns the remote cluster command connection for the specific address.
*/
private def clusterCore(address: Address): ActorRef =
context.actorFor(RootActorPath(address) / "system" / "cluster" / "core" / "daemon")
private def clusterCore(address: Address): ActorSelection =
context.actorSelection(RootActorPath(address) / "system" / "cluster" / "core" / "daemon")
val heartbeatSender = context.actorOf(Props[ClusterHeartbeatSender].
withDispatcher(UseDispatcher), name = "heartbeatSender")
@ -890,7 +891,7 @@ private[cluster] final class FirstSeedNodeProcess(seedNodes: immutable.IndexedSe
case JoinSeedNode
if (timeout.hasTimeLeft) {
// send InitJoin to remaining seed nodes (except myself)
remainingSeedNodes foreach { a context.actorFor(context.parent.path.toStringWithAddress(a)) ! InitJoin }
remainingSeedNodes foreach { a context.actorSelection(context.parent.path.toStringWithAddress(a)) ! InitJoin }
} else {
// no InitJoinAck received, initialize new cluster by joining myself
context.parent ! JoinTo(selfAddress)
@ -951,7 +952,7 @@ private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq
case JoinSeedNode
// send InitJoin to all seed nodes (except myself)
seedNodes.collect {
case a if a != selfAddress context.actorFor(context.parent.path.toStringWithAddress(a))
case a if a != selfAddress context.actorSelection(context.parent.path.toStringWithAddress(a))
} foreach { _ ! InitJoin }
case InitJoinAck(address)
// first InitJoinAck reply

View file

@ -7,7 +7,7 @@ import language.postfixOps
import scala.collection.immutable
import scala.concurrent.duration._
import akka.actor.{ ActorLogging, ActorRef, Address, Actor, RootActorPath, Props }
import akka.actor.{ ActorLogging, ActorRef, ActorSelection, Address, Actor, RootActorPath, Props }
import akka.cluster.ClusterEvent._
import akka.routing.MurmurHash
@ -113,13 +113,14 @@ private[cluster] final class ClusterHeartbeatSender extends Actor with ActorLogg
/**
* Looks up and returns the remote cluster heartbeat connection for the specific address.
*/
def heartbeatReceiver(address: Address): ActorRef =
context.actorFor(RootActorPath(address) / "system" / "cluster" / "heartbeatReceiver")
def heartbeatReceiver(address: Address): ActorSelection =
context.actorSelection(RootActorPath(address) / "system" / "cluster" / "heartbeatReceiver")
/**
* Looks up and returns the remote cluster heartbeat sender for the specific address.
*/
def heartbeatSender(address: Address): ActorRef = context.actorFor(self.path.toStringWithAddress(address))
def heartbeatSender(address: Address): ActorSelection =
context.actorSelection(self.path.toStringWithAddress(address))
def receive = {
case HeartbeatTick heartbeat()

View file

@ -157,7 +157,7 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto
sendGossip(address, MetricsGossipEnvelope(selfAddress, latestGossip, reply = true))
def sendGossip(address: Address, envelope: MetricsGossipEnvelope): Unit =
context.actorFor(self.path.toStringWithAddress(address)) ! envelope
context.actorSelection(self.path.toStringWithAddress(address)) ! envelope
def selectRandomNode(addresses: immutable.IndexedSeq[Address]): Option[Address] =
if (addresses.isEmpty) None else Some(addresses(ThreadLocalRandom.current nextInt addresses.size))

View file

@ -20,6 +20,8 @@ import akka.actor.Address
import akka.remote.RemoteActorRef
import java.util.concurrent.TimeoutException
import akka.actor.ActorSystemImpl
import akka.actor.ActorIdentity
import akka.actor.Identify
object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
@ -65,13 +67,19 @@ abstract class ClusterDeathWatchSpec
val path2 = RootActorPath(second) / "user" / "subject"
val path3 = RootActorPath(third) / "user" / "subject"
val watchEstablished = TestLatch(1)
val watchEstablished = TestLatch(2)
system.actorOf(Props(new Actor {
context.watch(context.actorFor(path2))
context.watch(context.actorFor(path3))
watchEstablished.countDown
context.actorSelection(path2) ! Identify(path2)
context.actorSelection(path3) ! Identify(path3)
def receive = {
case t: Terminated testActor ! t.actor.path
case ActorIdentity(`path2`, Some(ref))
context.watch(ref)
watchEstablished.countDown
case ActorIdentity(`path3`, Some(ref))
context.watch(ref)
watchEstablished.countDown
case Terminated(actor) testActor ! actor.path
}
}), name = "observer1")

View file

@ -83,7 +83,7 @@ abstract class RestartFirstSeedNodeSpec
enterBarrier("seed1-address-receiver-ready")
seedNode1Address = Cluster(seed1System).selfAddress
List(seed2, seed3) foreach { r
system.actorFor(RootActorPath(r) / "user" / "address-receiver") ! seedNode1Address
system.actorSelection(RootActorPath(r) / "user" / "address-receiver") ! seedNode1Address
expectMsg(5 seconds, "ok")
}
}

View file

@ -8,6 +8,7 @@ import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.atomic.AtomicReference
import org.scalatest.BeforeAndAfterEach
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
@ -35,6 +36,8 @@ import akka.remote.testkit.MultiNodeSpec
import akka.routing.FromConfig
import akka.testkit._
import akka.testkit.TestEvent._
import akka.actor.Identify
import akka.actor.ActorIdentity
/**
* This test is intended to be used as long running stress test
@ -553,9 +556,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
* Used for remote death watch testing
*/
class Watchee extends Actor {
def receive = {
case Ping sender ! Pong
}
def receive = Actor.emptyBehavior
}
/**
@ -623,9 +624,6 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
case class ChildrenCount(numberOfChildren: Int, numberOfChildRestarts: Int)
case object Reset
case object Ping
case object Pong
}
class StressMultiJvmNode1 extends StressSpec
@ -652,6 +650,8 @@ abstract class StressSpec
val settings = new Settings(system.settings.config)
import settings._
val identifyProbe = TestProbe()
var step = 0
var nbrUsedRoles = 0
@ -687,13 +687,16 @@ abstract class StressSpec
}
enterBarrier("result-aggregator-created-" + step)
runOn(roles.take(nbrUsedRoles): _*) {
phiObserver ! ReportTo(Some(clusterResultAggregator))
phiObserver ! ReportTo(clusterResultAggregator)
statsObserver ! Reset
statsObserver ! ReportTo(Some(clusterResultAggregator))
statsObserver ! ReportTo(clusterResultAggregator)
}
}
def clusterResultAggregator: ActorRef = system.actorFor(node(roles.head) / "user" / ("result" + step))
def clusterResultAggregator: Option[ActorRef] = {
system.actorSelection(node(roles.head) / "user" / ("result" + step)).tell(Identify(step), identifyProbe.ref)
identifyProbe.expectMsgType[ActorIdentity].ref
}
lazy val clusterResultHistory = system.actorOf(Props[ClusterResultHistory], "resultHistory")
@ -703,9 +706,12 @@ abstract class StressSpec
def awaitClusterResult(): Unit = {
runOn(roles.head) {
val r = clusterResultAggregator
clusterResultAggregator match {
case Some(r)
watch(r)
expectMsgPF(remaining) { case Terminated(a) if a.path == r.path true }
case None // ok, already terminated
}
}
enterBarrier("cluster-result-done-" + step)
}
@ -778,9 +784,9 @@ abstract class StressSpec
}
enterBarrier("watchee-created-" + step)
runOn(roles.head) {
system.actorFor(node(removeRole) / "user" / "watchee") ! Ping
expectMsg(Pong)
watch(lastSender)
system.actorSelection(node(removeRole) / "user" / "watchee").tell(Identify("watchee"), identifyProbe.ref)
val watchee = identifyProbe.expectMsgType[ActorIdentity].ref.get
watch(watchee)
}
enterBarrier("watch-estabilished-" + step)
@ -838,8 +844,9 @@ abstract class StressSpec
val returnValue = thunk
clusterResultAggregator !
ClusterResult(cluster.selfAddress, (System.nanoTime - startTime).nanos, cluster.readView.latestStats :- startStats)
clusterResultAggregator foreach {
_ ! ClusterResult(cluster.selfAddress, (System.nanoTime - startTime).nanos, cluster.readView.latestStats :- startStats)
}
returnValue
}
@ -904,7 +911,12 @@ abstract class StressSpec
}
def master: ActorRef = system.actorFor("/user/master-" + myself.name)
def masterName: String = "master-" + myself.name
def master: Option[ActorRef] = {
system.actorSelection("/user/" + masterName).tell(Identify("master"), identifyProbe.ref)
identifyProbe.expectMsgType[ActorIdentity].ref
}
def exerciseRouters(title: String, duration: FiniteDuration, batchInterval: FiniteDuration,
expectDroppedMessages: Boolean, tree: Boolean): Unit =
@ -940,13 +952,16 @@ abstract class StressSpec
}
def awaitWorkResult: WorkResult = {
val m = master
val workResult = expectMsgType[WorkResult]
log.info("{} result, [{}] jobs/s, retried [{}] of [{}] msg", m.path.name,
log.info("{} result, [{}] jobs/s, retried [{}] of [{}] msg", masterName,
workResult.jobsPerSecond.form,
workResult.retryCount, workResult.sendCount)
master match {
case Some(m)
watch(m)
expectMsgPF(remaining) { case Terminated(a) if a.path == m.path true }
case None // ok, already terminated
}
workResult
}
@ -1043,7 +1058,8 @@ abstract class StressSpec
"end routers that are running while nodes are joining" taggedAs LongRunningTest in within(30.seconds) {
runOn(roles.take(3): _*) {
val m = master
m.tell(End, testActor)
m must not be (None)
m.get.tell(End, testActor)
val workResult = awaitWorkResult
workResult.retryCount must be(0)
workResult.sendCount must be > (0L)
@ -1129,7 +1145,8 @@ abstract class StressSpec
"end routers that are running while nodes are removed" taggedAs LongRunningTest in within(30.seconds) {
runOn(roles.take(3): _*) {
val m = master
m.tell(End, testActor)
m must not be (None)
m.get.tell(End, testActor)
val workResult = awaitWorkResult
workResult.sendCount must be > (0L)
workResult.ackCount must be > (0L)

View file

@ -39,7 +39,7 @@ take over and a new singleton actor is created. For these failure scenarios ther
not be a graceful hand-over, but more than one active singletons is prevented by all
reasonable means. Some corner cases are eventually resolved by configurable timeouts.
You access the singleton actor with ``actorFor`` using the names you have specified when
You access the singleton actor with ``actorSelection`` using the names you have specified when
creating the ClusterSingletonManager. You can subscribe to cluster ``LeaderChanged`` or
``RoleLeaderChanged`` events to keep track of which node it is supposed to be running on.
Alternatively the singleton actor may broadcast its existence when it is started.
@ -78,7 +78,7 @@ This message will be sent over to the ``ClusterSingletonManager`` at the new lea
will be passed to the ``singletonProps`` factory when creating the new singleton instance.
With the names given above the path of singleton actor can be constructed by subscribing to
``RoleLeaderChanged`` cluster event and the actor reference is then looked up using ``actorFor``:
``RoleLeaderChanged`` cluster event and the actor reference is then looked up using ``actorSelection``:
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#singleton-proxy2

View file

@ -9,6 +9,7 @@ import akka.actor.Actor
import akka.actor.Actor.Receive
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.ActorSelection
import akka.actor.Address
import akka.actor.FSM
import akka.actor.Props
@ -229,7 +230,7 @@ class ClusterSingletonManagerIsStuck(message: String) extends AkkaException(mess
* is prevented by all reasonable means. Some corner cases are eventually
* resolved by configurable timeouts.
*
* You access the singleton actor with `actorFor` using the names you have
* You access the singleton actor with `actorSelection` using the names you have
* specified when creating the ClusterSingletonManager. You can subscribe to
* [[akka.cluster.ClusterEvent.LeaderChanged]] or
* [[akka.cluster.ClusterEvent.RoleLeaderChanged]] to keep track of which node
@ -385,7 +386,7 @@ class ClusterSingletonManager(
super.postStop()
}
def peer(at: Address): ActorRef = context.actorFor(self.path.toStringWithAddress(at))
def peer(at: Address): ActorSelection = context.actorSelection(self.path.toStringWithAddress(at))
def getNextLeaderChanged(): Unit =
if (leaderChangedReceived) {

View file

@ -25,6 +25,7 @@ import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
import akka.testkit.TestEvent._
import akka.actor.Terminated
import akka.actor.ActorSelection
object ClusterSingletonManagerChaosSpec extends MultiNodeConfig {
val controller = role("controller")
@ -104,8 +105,8 @@ class ClusterSingletonManagerChaosSpec extends MultiNodeSpec(ClusterSingletonMan
}
}
def echo(leader: RoleName): ActorRef =
system.actorFor(RootActorPath(node(leader).address) / "user" / "singleton" / "echo")
def echo(leader: RoleName): ActorSelection =
system.actorSelection(RootActorPath(node(leader).address) / "user" / "singleton" / "echo")
def verify(leader: RoleName): Unit = {
enterBarrier("before-" + leader.name + "-verified")

View file

@ -24,6 +24,9 @@ import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._
import akka.testkit.TestEvent._
import akka.actor.Terminated
import akka.actor.Identify
import akka.actor.ActorIdentity
import akka.actor.ActorSelection
object ClusterSingletonManagerSpec extends MultiNodeConfig {
val controller = role("controller")
@ -157,11 +160,11 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
def receive = {
case state: CurrentClusterState leaderAddress = state.leader
case LeaderChanged(leader) leaderAddress = leader
case other consumer foreach { _ forward other }
case other consumer foreach { _.tell(other, sender) }
}
def consumer: Option[ActorRef] =
leaderAddress map (a context.actorFor(RootActorPath(a) /
def consumer: Option[ActorSelection] =
leaderAddress map (a context.actorSelection(RootActorPath(a) /
"user" / "singleton" / "consumer"))
}
//#singleton-proxy
@ -181,11 +184,11 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
def receive = {
case state: CurrentClusterState leaderAddress = state.roleLeader(role)
case RoleLeaderChanged(r, leader) if (r == role) leaderAddress = leader
case other consumer foreach { _ forward other }
case other consumer foreach { _.tell(other, sender) }
}
def consumer: Option[ActorRef] =
leaderAddress map (a context.actorFor(RootActorPath(a) /
def consumer: Option[ActorSelection] =
leaderAddress map (a context.actorSelection(RootActorPath(a) /
"user" / "singleton" / "consumer"))
}
//#singleton-proxy2
@ -208,6 +211,8 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
override def initialParticipants = roles.size
val identifyProbe = TestProbe()
//#sort-cluster-roles
// Sort the roles in the order used by the cluster.
lazy val sortedWorkerNodes: immutable.IndexedSeq[RoleName] = {
@ -220,7 +225,10 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
}
//#sort-cluster-roles
def queue: ActorRef = system.actorFor(node(controller) / "user" / "queue")
def queue: ActorRef = {
system.actorSelection(node(controller) / "user" / "queue").tell(Identify("queue"), identifyProbe.ref)
identifyProbe.expectMsgType[ActorIdentity].ref.get
}
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
@ -241,8 +249,8 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
//#create-singleton-manager
}
def consumer(leader: RoleName): ActorRef =
system.actorFor(RootActorPath(node(leader).address) / "user" / "singleton" / "consumer")
def consumer(leader: RoleName): ActorSelection =
system.actorSelection(RootActorPath(node(leader).address) / "user" / "singleton" / "consumer")
def verify(leader: RoleName, msg: Int, expectedCurrent: Int): Unit = {
enterBarrier("before-" + leader.name + "-verified")
@ -340,10 +348,14 @@ class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerS
verify(newLeaderRole, msg = 7, expectedCurrent = 6)
runOn(leaveRole) {
val singleton = system.actorFor("/user/singleton")
system.actorSelection("/user/singleton").tell(Identify("singleton"), identifyProbe.ref)
identifyProbe.expectMsgPF() {
case ActorIdentity("singleton", None) // already terminated
case ActorIdentity("singleton", Some(singleton))
watch(singleton)
expectMsgType[Terminated].actor must be(singleton)
}
}
enterBarrier("after-leave")
}

View file

@ -5,7 +5,6 @@
package akka.contrib.pattern
import language.postfixOps
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
@ -18,6 +17,8 @@ import scala.concurrent.duration._
import akka.actor.FSM
import akka.actor.ActorRef
import akka.testkit.TestProbe
import akka.actor.ActorIdentity
import akka.actor.Identify
object ReliableProxySpec extends MultiNodeConfig {
val local = role("local")
@ -68,7 +69,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
//#demo
import akka.contrib.pattern.ReliableProxy
target = system.actorFor(node(remote) / "user" / "echo")
system.actorSelection(node(remote) / "user" / "echo") ! Identify("echo")
target = expectMsgType[ActorIdentity].ref.get
proxy = system.actorOf(Props(new ReliableProxy(target, 100.millis)), "proxy")
//#demo
proxy ! FSM.SubscribeTransitionCallBack(testActor)

View file

@ -194,34 +194,43 @@ Looking up Actors by Concrete Path
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
In addition, actor references may be looked up using the
:meth:`ActorSystem.actorFor` method, which returns a local or remote actor
reference. The reference can be reused for communicating with said actor during
the whole lifetime of the actor. In the case of a local actor reference, the
named actor needs to exist before the lookup, or else the acquired reference
will be an :class:`EmptyLocalActorRef`. This will be true even if an actor with
that exact path is created after acquiring the actor reference. For remote actor
references acquired with `actorFor` the behaviour is different and sending messages
to such a reference will under the hood look up the actor by path on the remote
system for every message send.
:meth:`ActorSystem.actorSelection` method. The selection can be used for
communicating with said actor and the actor corresponding to the selection
is looked up when delivering each message.
To acquire an :class:`ActorRef` that is bound to the life-cycle of a specific actor
you need to send a message, such as the built-in :class:`Identify` message, to the actor
and use the ``sender`` reference of a reply from the actor.
.. note::
``actorFor`` is deprecated in favor of ``actorSelection`` because actor references
acquired with ``actorFor`` behave differently for local and remote actors.
In the case of a local actor reference, the named actor needs to exist before the
lookup, or else the acquired reference will be an :class:`EmptyLocalActorRef`.
This will be true even if an actor with that exact path is created after acquiring
the actor reference. For remote actor references acquired with `actorFor` the
behaviour is different and sending messages to such a reference will under the hood
look up the actor by path on the remote system for every message send.
Absolute vs. Relative Paths
```````````````````````````
In addition to :meth:`ActorSystem.actorFor` there is also
:meth:`ActorContext.actorFor`, which is available inside any actor as
``context.actorFor``. This yields an actor reference much like its twin on
In addition to :meth:`ActorSystem.actorSelection` there is also
:meth:`ActorContext.actorSelection`, which is available inside any actor as
``context.actorSelection``. This yields an actor selection much like its twin on
:class:`ActorSystem`, but instead of looking up the path starting from the root
of the actor tree it starts out on the current actor. Path elements consisting
of two dots (``".."``) may be used to access the parent actor. You can for
example send a message to a specific sibling::
context.actorFor("../brother") ! msg
context.actorSelection("../brother") ! msg
Absolute paths may of course also be looked up on `context` in the usual way, i.e.
.. code-block:: scala
context.actorFor("/user/serviceA") ! msg
context.actorSelection("/user/serviceA") ! msg
will work as expected.
@ -249,10 +258,10 @@ extracting the sender references, and then watch all discovered concrete
actors. This scheme of resolving a selection may be improved upon in a future
release.
.. _actorOf-vs-actorFor:
.. _actorOf-vs-actorSelection:
Summary: ``actorOf`` vs. ``actorFor``
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Summary: ``actorOf`` vs. ``actorSelection`` vs. ``actorFor``
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. note::
@ -263,8 +272,12 @@ Summary: ``actorOf`` vs. ``actorFor``
child of the context on which this method is invoked (which may be any
actor or actor system).
- ``actorFor`` only ever looks up an existing actor, i.e. does not create
one.
- ``actorSelection`` only ever looks up existing actors when messages are
delivered, i.e. does not create actors, or verify existence of actors
when the selection is created.
- ``actorFor`` (deprecated in favor of actorSelection) only ever looks up an
existing actor, i.e. does not create one.
Actor Reference and Path Equality
---------------------------------

View file

@ -176,12 +176,13 @@ public class TypedActorDocTestBase {
@Test public void proxyAnyActorRef() {
try {
final ActorRef actorRefToRemoteActor = system.deadLetters();
//#typed-actor-remote
Squarer typedActor =
TypedActor.get(system).
typedActorOf(
new TypedProps<Squarer>(Squarer.class),
system.actorFor("akka://SomeSystem@somehost:2552/user/some/foobar")
actorRefToRemoteActor
);
//Use "typedActor" as a FooBar
//#typed-actor-remote

View file

@ -15,6 +15,7 @@ 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
@ -31,6 +32,12 @@ import akka.japi.Procedure;
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;
@ -58,6 +65,8 @@ import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import org.junit.Test;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import scala.Option;
import java.lang.Object;
import java.util.Iterator;
@ -65,6 +74,19 @@ import akka.pattern.Patterns;
public class UntypedActorDocTestBase {
private static ActorSystem system;
@BeforeClass
public static void beforeAll() {
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@Test
public void createProps() {
//#creating-props-config
@ -96,86 +118,71 @@ public class UntypedActorDocTestBase {
@Test
public void contextActorOf() {
//#context-actorOf
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class), "myactor");
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class), "myactor2");
//#context-actorOf
myActor.tell("test", null);
system.shutdown();
}
@Test
public void constructorActorOf() {
ActorSystem system = ActorSystem.create("MySystem");
//#creating-constructor
// allows passing in arguments to the MyActor constructor
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
public UntypedActor create() {
return new MyActor("...");
}
}), "myactor");
}), "myactor3");
//#creating-constructor
myActor.tell("test", null);
system.shutdown();
}
@Test
public void propsActorOf() {
ActorSystem system = ActorSystem.create("MySystem");
//#creating-props
ActorRef myActor = system.actorOf(
new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), "myactor");
new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), "myactor4");
//#creating-props
myActor.tell("test", null);
system.shutdown();
}
@Test
public void usingAsk() throws Exception {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
public UntypedActor create() {
return new MyAskActor();
}
}), "myactor");
}), "myactor5");
//#using-ask
Future<Object> future = Patterns.ask(myActor, "Hello", 1000);
Object result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
//#using-ask
system.shutdown();
}
@Test
public void receiveTimeout() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(MyReceivedTimeoutUntypedActor.class));
myActor.tell("Hello", null);
system.shutdown();
}
@Test
public void usePoisonPill() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class));
//#poison-pill
myActor.tell(PoisonPill.getInstance(), null);
//#poison-pill
system.shutdown();
}
@Test
public void useKill() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef victim = system.actorOf(new Props(MyUntypedActor.class));
//#kill
victim.tell(Kill.getInstance(), null);
//#kill
system.shutdown();
}
@Test
public void useBecome() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
public UntypedActor create() {
return new HotSwapActor();
@ -184,21 +191,24 @@ public class UntypedActorDocTestBase {
myActor.tell("foo", null);
myActor.tell("bar", null);
myActor.tell("bar", null);
system.shutdown();
}
@Test
public void useWatch() throws Exception {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(WatchActor.class));
Future<Object> future = Patterns.ask(myActor, "kill", 1000);
assert Await.result(future, Duration.create("1 second")).equals("finished");
system.shutdown();
}
@Test
public void useIdentify() throws Exception {
ActorRef a = system.actorOf(new Props(MyUntypedActor.class), "another");
ActorRef b = system.actorOf(new Props(Follower.class));
system.stop(a);
}
@Test
public void usePatternsGracefulStop() throws Exception {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef actorRef = system.actorOf(new Props(MyUntypedActor.class));
//#gracefulStop
try {
@ -210,7 +220,6 @@ public class UntypedActorDocTestBase {
// the actor wasn't stopped within 5 seconds
}
//#gracefulStop
system.shutdown();
}
class Result {
@ -225,7 +234,6 @@ public class UntypedActorDocTestBase {
@Test
public void usePatternsAskPipe() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef actorA = system.actorOf(new Props(MyUntypedActor.class));
ActorRef actorB = system.actorOf(new Props(MyUntypedActor.class));
ActorRef actorC = system.actorOf(new Props(MyUntypedActor.class));
@ -251,7 +259,6 @@ public class UntypedActorDocTestBase {
pipe(transformed, system.dispatcher()).to(actorC);
//#ask-pipe
system.shutdown();
}
public static class MyActor extends UntypedActor {
@ -399,4 +406,40 @@ public class UntypedActorDocTestBase {
}
//#watch
static
//#identify
public class Follower extends UntypedActor {
String identifyId = "1";
{
ActorSelection selection =
getContext().actorSelection("/user/another");
selection.tell(new Identify(identifyId), getSelf());
}
ActorRef another;
@Override
public void onReceive(Object message) {
if (message instanceof ActorIdentity) {
ActorIdentity identity = (ActorIdentity) message;
if (identity.correlationId().equals(identifyId)) {
ActorRef ref = identity.getRef();
if (ref == null)
getContext().stop(getSelf());
else {
another = ref;
getContext().watch(another);
}
}
} else if (message instanceof Terminated) {
final Terminated t = (Terminated) message;
if (t.getActor().equals(another)) {
getContext().stop(getSelf());
}
} else {
unhandled(message);
}
}
}
//#identify
}

View file

@ -49,10 +49,10 @@ public class SerializationDocTestBase {
//#my-own-serializer
@Test public void serializeActorRefs() {
final ActorSystem theActorSystem =
final ExtendedActorSystem extendedSystem = (ExtendedActorSystem)
ActorSystem.create("whatever");
final ActorRef theActorRef =
theActorSystem.deadLetters(); // Of course this should be you
extendedSystem.deadLetters(); // Of course this should be you
//#actorref-serializer
// Serialize
@ -63,10 +63,10 @@ public class SerializationDocTestBase {
// Deserialize
// (beneath fromBinary)
final ActorRef deserializedActorRef = theActorSystem.actorFor(identifier);
final ActorRef deserializedActorRef = extendedSystem.provider().resolveActorRef(identifier);
// Then just use the ActorRef
//#actorref-serializer
theActorSystem.shutdown();
extendedSystem.shutdown();
}
static

View file

@ -16,6 +16,7 @@ import akka.actor.ActorKilledException;
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.actor.UntypedActor;
@ -326,11 +327,12 @@ public class TestKitDocTest {
@Test
public void demonstrateWatch() {
final ActorRef target = system.actorFor("/buh");
final ActorRef target = system.actorOf(new Props(MyActor.class));
//#test-probe-watch
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
probe.watch(target);
target.tell(PoisonPill.getInstance(), null);
final Terminated msg = probe.expectMsgClass(Terminated.class);
assertEquals(msg.getActor(), target);
}};

View file

@ -70,17 +70,24 @@ reference file for more information:
Looking up Remote Actors
^^^^^^^^^^^^^^^^^^^^^^^^
``actorFor(path)`` will obtain an ``ActorRef`` to an Actor on a remote node::
``actorSelection(path)`` will obtain an ``ActorSelection`` to an Actor on a remote node::
ActorRef actor = context.actorFor("akka.tcp://app@10.0.0.1:2552/user/serviceA/worker");
ActorSelection selection =
context.actorSelection("akka.tcp://app@10.0.0.1:2552/user/serviceA/worker");
As you can see from the example above the following pattern is used to find an ``ActorRef`` on a remote node::
As you can see from the example above the following pattern is used to find an actor on a remote node::
akka.<protocol>://<actorsystemname>@<hostname>:<port>/<actor path>
Once you obtained a reference to the actor you can interact with it they same way you would with a local actor, e.g.::
Once you obtained a selection to the actor you can interact with it they same way you would with a local actor, e.g.::
actor.tell("Pretty awesome feature", getSelf());
selection.tell("Pretty awesome feature", getSelf());
To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
send a message to the selection and use the ``getSender`` reference of the reply from
the actor. There is a built-in ``Identify`` message that all Actors will understand
and automatically reply to with a ``ActorIdentity`` message containing the
:class:`ActorRef`.
.. note::
@ -264,9 +271,9 @@ and it is created from an actor system using the aforementioned clients confi
.. includecode:: ../../../akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApplication.java
:include: setup
Requests which come in via ``doSomething`` will be sent to the client actor
along with the reference which was looked up earlier. Observe how the actor
system name using in ``actorFor`` matches the remote systems name, as do IP
Requests which come in via ``doSomething`` will be sent to the client actor,
which will use the actor reference that was identified earlier. Observe how the actor
system name using in ``actorSelection`` matches the remote systems name, as do IP
and port number. Top-level actors are always created below the ``"/user"``
guardian, which supervises them.

View file

@ -206,11 +206,11 @@ Proxying
--------
You can use the ``typedActorOf`` that takes a TypedProps and an ActorRef to proxy the given ActorRef as a TypedActor.
This is usable if you want to communicate remotely with TypedActors on other machines, just look them up with ``actorFor`` and pass the ``ActorRef`` to ``typedActorOf``.
This is usable if you want to communicate remotely with TypedActors on other machines, just pass the ``ActorRef`` to ``typedActorOf``.
Lookup & Remoting
-----------------
Since ``TypedActors`` are backed by ``Akka Actors``, you can use ``actorFor`` together with ``typedActorOf`` to proxy ``ActorRefs`` potentially residing on remote nodes.
Since ``TypedActors`` are backed by ``Akka Actors``, you can use ``typedActorOf`` to proxy ``ActorRefs`` potentially residing on remote nodes.
.. includecode:: code/docs/actor/TypedActorDocTestBase.java#typed-actor-remote

View file

@ -27,8 +27,7 @@ Creating Actors
Since Akka enforces parental supervision every actor is supervised and
(potentially) the supervisor of its children, it is advisable that you
familiarize yourself with :ref:`actor-systems` and :ref:`supervision` and it
may also help to read :ref:`actorOf-vs-actorFor` (the whole of
:ref:`addressing` is recommended reading in any case).
may also help to read :ref:`addressing`.
Defining an Actor class
-----------------------
@ -264,8 +263,10 @@ sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
:obj:`ActorSystem`.
Identifying Actors
==================
.. _actorSelection-java:
Identifying Actors via Actor Selection
======================================
As described in :ref:`addressing`, each actor has a unique logical path, which
is obtained by following the chain of actors from child to parent until
@ -274,11 +275,13 @@ differ if the supervision chain includes any remote supervisors. These paths
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:
actors may look up other actors by specifying absolute or relative
paths—logical or physical—and receive back an :class:`ActorRef` with the
paths—logical or physical—and receive back an :class:`ActorSelection` with the
result::
getContext().actorFor("/user/serviceA/actor") // will look up this absolute path
getContext().actorFor("../joe") // will look up sibling beneath same supervisor
// will look up this absolute path
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
that it is split on ``/`` into path elements. If the path starts with ``/``, it
@ -289,18 +292,43 @@ currently traversed actor, otherwise it will step “down” to the named child.
It should be noted that the ``..`` in actor paths here always means the logical
structure, i.e. the supervisor.
If the path being looked up does not exist, a special actor reference is
returned which behaves like the actor systems dead letter queue but retains
its identity (i.e. the path which was looked up).
The path elements of an actor selection may contain wildcard patterns allowing for
broadcasting of messages to that section::
Remote actor addresses may also be looked up, if remoting is enabled::
// 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("../*");
getContext().actorFor("akka.tcp://app@otherhost:1234/user/serviceB")
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
does not match any actors the message will be dropped.
These look-ups return a (possibly remote) actor reference immediately, so you
will have to send to it and await a reply in order to verify that ``serviceB``
is actually reachable and running. An example demonstrating actor look-up is
given in :ref:`remote-lookup-sample-java`.
To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
send a message to the selection and use the ``getSender`` reference of the reply from
the actor. There is a built-in ``Identify`` message that all Actors will understand
and automatically reply to with a ``ActorIdentity`` message containing the
:class:`ActorRef`.
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
:include: identify-imports,identify
Remote actor addresses may also be looked up, if :ref:`remoting <remoting-java>` is enabled::
getContext().actorSelection("akka.tcp://app@otherhost:1234/user/serviceB");
An example demonstrating remote actor look-up is given in :ref:`remote-lookup-sample-java`.
.. note::
``actorFor`` is deprecated in favor of ``actorSelection`` because actor references
acquired with ``actorFor`` behave differently for local and remote actors.
In the case of a local actor reference, the named actor needs to exist before the
lookup, or else the acquired reference will be an :class:`EmptyLocalActorRef`.
This will be true even if an actor with that exact path is created after acquiring
the actor reference. For remote actor references acquired with `actorFor` the
behaviour is different and sending messages to such a reference will under the hood
look up the actor by path on the remote system for every message send.
Messages and immutability
=========================

View file

@ -152,6 +152,33 @@ available via the ``inbound`` boolean field of the event.
New configuration settings are also available, see the remoting documentation for more detail: :ref:`remoting-scala`
.. _migration_2.2_actorSelection:
Use ``actorSelection`` instead of ``actorFor``
==============================================
``actorFor`` is deprecated in favor of ``actorSelection`` because actor references
acquired with ``actorFor`` behave differently for local and remote actors.
In the case of a local actor reference, the named actor needs to exist before the
lookup, or else the acquired reference will be an :class:`EmptyLocalActorRef`.
This will be true even if an actor with that exact path is created after acquiring
the actor reference. For remote actor references acquired with `actorFor` the
behaviour is different and sending messages to such a reference will under the hood
look up the actor by path on the remote system for every message send.
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
does not match any actors the message will be dropped.
To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
send a message to the selection and use the ``sender`` reference of the reply from
the actor. There is a built-in ``Identify`` message that all Actors will understand
and automatically reply to with a ``ActorIdentity`` message containing the
:class:`ActorRef`.
Read more about ``actorSelection`` in :ref:`docs for Java <actorSelection-java>` or
:ref:`docs for Scala <actorSelection-scala>`.
ActorRef equality and sending to remote actors
==============================================
@ -187,9 +214,32 @@ messages to remote deployed children of a restarted parent.
This may also have implications if you compare the ``ActorRef`` received in a ``Terminated`` message
with an expected ``ActorRef``.
.. _migration_2.2_actorSelection:
The following will not match::
Use ``actorSelection`` instead of ``actorFor``
==============================================
val ref = context.actorFor("akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName")
FIXME: ticket #3074
def receive = {
case Terminated(`ref`) => // ...
}
Instead, use actorSelection followed by identify request, and watch the verified actor reference::
val selection = context.actorSelection("akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName")
selection ! Identify(None)
var ref: ActorRef = _
def receive = {
case ActorIdentity(_, Some(actorRef)) =>
ref = actorRef
context watch ref
case ActorIdentity(_, None) => // not alive
case Terminated(`ref`) => // ...
}
Use ``watch`` instead of ``isTerminated``
=========================================
``ActorRef.isTerminated`` is deprecated in favor of ``ActorContext.watch`` because
``isTerminated`` behaves differently for local and remote actors.

View file

@ -27,8 +27,7 @@ Creating Actors
Since Akka enforces parental supervision every actor is supervised and
(potentially) the supervisor of its children, it is advisable that you
familiarize yourself with :ref:`actor-systems` and :ref:`supervision` and it
may also help to read :ref:`actorOf-vs-actorFor` (the whole of
:ref:`addressing` is recommended reading in any case).
may also help to read :ref:`addressing`.
Defining an Actor class
-----------------------
@ -375,8 +374,10 @@ to run after message queuing has been disabled for this actor, i.e. messages
sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
:obj:`ActorSystem`.
Identifying Actors
==================
.. _actorSelection-scala:
Identifying Actors via Actor Selection
======================================
As described in :ref:`addressing`, each actor has a unique logical path, which
is obtained by following the chain of actors from child to parent until
@ -385,11 +386,13 @@ differ if the supervision chain includes any remote supervisors. These paths
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:
actors may look up other actors by specifying absolute or relative
paths—logical or physical—and receive back an :class:`ActorRef` with the
paths—logical or physical—and receive back an :class:`ActorSelection` with the
result::
context.actorFor("/user/serviceA/aggregator") // will look up this absolute path
context.actorFor("../joe") // will look up sibling beneath same supervisor
// will look up this absolute path
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
that it is split on ``/`` into path elements. If the path starts with ``/``, it
@ -400,18 +403,42 @@ currently traversed actor, otherwise it will step “down” to the named child.
It should be noted that the ``..`` in actor paths here always means the logical
structure, i.e. the supervisor.
If the path being looked up does not exist, a special actor reference is
returned which behaves like the actor systems dead letter queue but retains
its identity (i.e. the path which was looked up).
The path elements of an actor selection may contain wildcard patterns allowing for
broadcasting of messages to that section::
Remote actor addresses may also be looked up, if remoting is enabled::
// 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("../*")
context.actorFor("akka.tcp://app@otherhost:1234/user/serviceB")
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
does not match any actors the message will be dropped.
These look-ups return a (possibly remote) actor reference immediately, so you
will have to send to it and await a reply in order to verify that ``serviceB``
is actually reachable and running. An example demonstrating actor look-up is
given in :ref:`remote-lookup-sample-scala`.
To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
send a message to the selection and use the ``sender`` reference of the reply from
the actor. There is a built-in ``Identify`` message that all Actors will understand
and automatically reply to with a ``ActorIdentity`` message containing the
:class:`ActorRef`.
.. includecode:: code/docs/actor/ActorDocSpec.scala#identify
Remote actor addresses may also be looked up, if :ref:`remoting <remoting-scala>` is enabled::
context.actorSelection("akka.tcp://app@otherhost:1234/user/serviceB")
An example demonstrating actor look-up is given in :ref:`remote-lookup-sample-scala`.
.. note::
``actorFor`` is deprecated in favor of ``actorSelection`` because actor references
acquired with ``actorFor`` behaves different for local and remote actors.
In the case of a local actor reference, the named actor needs to exist before the
lookup, or else the acquired reference will be an :class:`EmptyLocalActorRef`.
This will be true even if an actor with that exact path is created after acquiring
the actor reference. For remote actor references acquired with `actorFor` the
behaviour is different and sending messages to such a reference will under the hood
look up the actor by path on the remote system for every message send.
Messages and immutability
=========================

View file

@ -327,6 +327,37 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
expectMsg("finished")
}
"using Identify" in {
//#identify
import akka.actor.{ Actor, Props, Identify, ActorIdentity, Terminated }
class Follower extends Actor {
val identifyId = 1
context.actorSelection("/user/another") ! Identify(identifyId)
def receive = {
case ActorIdentity(`identifyId`, Some(ref))
context.watch(ref)
context.become(active(ref))
case ActorIdentity(`identifyId`, None) context.stop(self)
}
def active(another: ActorRef): Actor.Receive = {
case Terminated(`another`) context.stop(self)
}
}
//#identify
val a = system.actorOf(Props(new Actor {
def receive = Actor.emptyBehavior
}))
val b = system.actorOf(Props(new Follower))
watch(b)
system.stop(a)
expectMsgType[akka.actor.Terminated].actor must be === b
}
"using pattern gracefulStop" in {
val actorRef = system.actorOf(Props[MyActor])
//#gracefulStop
@ -386,9 +417,9 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
lastSender must be === actor
actor ! me
expectMsg("reply")
lastSender must be === system.actorFor("/user")
lastSender.path.elements.mkString("/", "/", "") must be === "/user"
expectMsg("reply")
lastSender must be === system.actorFor("/user")
lastSender.path.elements.mkString("/", "/", "") must be === "/user"
}
"using ActorDSL outside of akka.actor package" in {

View file

@ -4,17 +4,15 @@
package docs.actor
import language.postfixOps
//#imports
import scala.concurrent.{ Promise, Future, Await }
import scala.concurrent.duration._
import akka.actor.{ ActorContext, TypedActor, TypedProps }
//#imports
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
import org.scalatest.matchers.MustMatchers
import akka.testkit._
//#typed-actor-impl
import java.lang.String.{ valueOf println } //Mr funny man avoids printing to stdout AND keeping docs alright
import akka.actor.ActorRef
//#typed-actor-iface
trait Squarer {
@ -46,8 +44,6 @@ class SquarerImpl(val name: String) extends Squarer {
def squareNow(i: Int): Int = i * i
//#typed-actor-impl-methods
}
//#typed-actor-impl
import java.lang.String.{ valueOf println } //Mr funny man avoids printing to stdout AND keeping docs alright
//#typed-actor-supercharge
trait Foo {
def doFoo(times: Int): Unit = println("doFoo(" + times + ")")
@ -145,12 +141,13 @@ class TypedActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
}
"proxy any ActorRef" in {
val actorRefToRemoteActor: ActorRef = system.deadLetters
//#typed-actor-remote
val typedActor: Foo with Bar =
TypedActor(system).
typedActorOf(
TypedProps[FooBar],
system.actorFor("akka://SomeSystem@somehost:2552/user/some/foobar"))
actorRefToRemoteActor)
//Use "typedActor" as a FooBar
//#typed-actor-remote
}

View file

@ -10,13 +10,15 @@ import akka.routing.RoundRobinRouter
import akka.testkit._
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
import akka.actor.ActorPath
object RouterViaProgramDocSpec {
case class Message1(nbr: Int)
case class Reply1(name: String, m: Message1)
class ExampleActor1 extends Actor {
def receive = {
case m @ Message1(nbr) sender ! ((self, m))
case m @ Message1(nbr) sender ! Reply1(self.path.name, m)
}
}
@ -43,9 +45,9 @@ class RouterViaProgramDocSpec extends AkkaSpec with ImplicitSender {
1 to 6 foreach { i router ! Message1(i) }
val received = receiveN(6, 5.seconds.dilated)
1 to 6 foreach { i
val expectedActor = system.actorFor(routees((i - 1) % routees.length))
val expectedName = (routees((i - 1) % routees.length)).split("/").last
val expectedMsg = Message1(i)
received must contain[AnyRef]((expectedActor, expectedMsg))
received must contain[AnyRef](Reply1(expectedName, expectedMsg))
}
}

View file

@ -157,7 +157,7 @@ package docs.serialization {
"demonstrate serialization of ActorRefs" in {
val theActorRef: ActorRef = system.deadLetters
val theActorSystem: ActorSystem = system
val extendedSystem: ExtendedActorSystem = system.asInstanceOf[ExtendedActorSystem]
//#actorref-serializer
// Serialize
@ -168,7 +168,7 @@ package docs.serialization {
// Deserialize
// (beneath fromBinary)
val deserializedActorRef = theActorSystem actorFor identifier
val deserializedActorRef = extendedSystem.provider.resolveActorRef(identifier)
// Then just use the ActorRef
//#actorref-serializer
@ -182,7 +182,7 @@ package docs.serialization {
}
def serializeTo(ref: ActorRef, remote: Address): String =
ref.path.toSerializationFormatWithAddress(ExternalAddress(theActorSystem).addressFor(remote))
ref.path.toSerializationFormatWithAddress(ExternalAddress(extendedSystem).addressFor(remote))
//#external-address
}

View file

@ -208,10 +208,13 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
"demonstrate probe watch" in {
import akka.testkit.TestProbe
val target = system.actorFor("/buh")
val target = system.actorOf(Props(new Actor {
def receive = Actor.emptyBehavior
}))
//#test-probe-watch
val probe = TestProbe()
probe watch target
target ! PoisonPill
probe.expectMsgType[Terminated].actor must be(target)
//#test-probe-watch
}

View file

@ -69,7 +69,7 @@ Types of Remote Interaction
Akka has two ways of using remoting:
* Lookup : used to look up an actor on a remote node with ``actorFor(path)``
* Lookup : used to look up an actor on a remote node with ``actorSelection(path)``
* Creation : used to create an actor on a remote node with ``actorOf(Props(...), actorName)``
In the next sections the two alternatives are described in detail.
@ -77,17 +77,24 @@ In the next sections the two alternatives are described in detail.
Looking up Remote Actors
^^^^^^^^^^^^^^^^^^^^^^^^
``actorFor(path)`` will obtain an ``ActorRef`` to an Actor on a remote node, e.g.::
``actorSelection(path)`` will obtain an ``ActorSelection`` to an Actor on a remote node, e.g.::
val actor = context.actorFor("akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName")
val selection =
context.actorSelection("akka.tcp://actorSystemName@10.0.0.1:2552/user/actorName")
As you can see from the example above the following pattern is used to find an ``ActorRef`` on a remote node::
As you can see from the example above the following pattern is used to find an actor on a remote node::
akka.<protocol>://<actor system>@<hostname>:<port>/<actor path>
Once you obtained a reference to the actor you can interact with it they same way you would with a local actor, e.g.::
Once you obtained a selection to the actor you can interact with it they same way you would with a local actor, e.g.::
actor ! "Pretty awesome feature"
selection ! "Pretty awesome feature"
To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
send a message to the selection and use the ``sender`` reference of the reply from
the actor. There is a built-in ``Identify`` message that all Actors will understand
and automatically reply to with a ``ActorIdentity`` message containing the
:class:`ActorRef`.
.. note::
@ -271,9 +278,9 @@ and it is created from an actor system using the aforementioned clients confi
.. includecode:: ../../../akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/LookupApplication.scala
:include: setup
Requests which come in via ``doSomething`` will be sent to the client actor
along with the reference which was looked up earlier. Observe how the actor
system name using in ``actorFor`` matches the remote systems name, as do IP
Requests which come in via ``doSomething`` will be sent to the client actor,
which will use the actor reference that was identified earlier. Observe how the actor
system name using in ``actorSelection`` matches the remote systems name, as do IP
and port number. Top-level actors are always created below the ``"/user"``
guardian, which supervises them.

View file

@ -204,7 +204,7 @@ Proxying
--------
You can use the ``typedActorOf`` that takes a TypedProps and an ActorRef to proxy the given ActorRef as a TypedActor.
This is usable if you want to communicate remotely with TypedActors on other machines, just look them up with ``actorFor`` and pass the ``ActorRef`` to ``typedActorOf``.
This is usable if you want to communicate remotely with TypedActors on other machines, just pass the ``ActorRef`` to ``typedActorOf``.
.. note::
@ -213,7 +213,7 @@ This is usable if you want to communicate remotely with TypedActors on other mac
Lookup & Remoting
-----------------
Since ``TypedActors`` are backed by ``Akka Actors``, you can use ``actorFor`` together with ``typedActorOf`` to proxy ``ActorRefs`` potentially residing on remote nodes.
Since ``TypedActors`` are backed by ``Akka Actors``, you can use ``typedActorOf`` to proxy ``ActorRefs`` potentially residing on remote nodes.
.. includecode:: code/docs/actor/TypedActorDocSpec.scala#typed-actor-remote

View file

@ -516,10 +516,10 @@ in case classes with proper type information. But one particular useful feature
of Akka actors is that they have a stable identity by which they can be found,
a unique name. This name is represented as a :class:`String` and naturally does
not bear any type information concerning the actors channels. Thus, when
looking up an actor with ``system.actorFor(...)`` you will only get an untyped
:class:`ActorRef` and not a channel reference. This :class:`ActorRef` can of
course manually be wrapped in a channel reference bearing the desired channels,
but this is not a type-safe operation.
looking up an actor with ``system.actorSelection(...)`` followed by an ``Identify``
request you will only get an untyped :class:`ActorRef` and not a channel reference.
This :class:`ActorRef` can of course manually be wrapped in a channel reference
bearing the desired channels, but this is not a type-safe operation.
The solution in this case must be a runtime check. There is an operation to
“narrow” an :class:`ActorRef` to a channel reference of given type, which

View file

@ -60,7 +60,8 @@ trait DurableMessageSerialization { this: DurableMessageQueue ⇒
*/
def deserialize(bytes: Array[Byte]): Envelope = {
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = system.actorFor(refProtocol.getPath)
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef =
system.provider.resolveActorRef(refProtocol.getPath)
val durableMessage = RemoteMessageProtocol.parseFrom(bytes)
val message = MessageSerializer.deserialize(system, durableMessage.getMessage)

View file

@ -354,7 +354,7 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
* return that as an ActorPath for easy composition:
*
* {{{
* val serviceA = system.actorFor(node("master") / "user" / "serviceA")
* val serviceA = system.actorSelection(node("master") / "user" / "serviceA")
* }}}
*/
def node(role: RoleName): ActorPath = RootActorPath(testConductor.getAddressFor(role).await)

View file

@ -9,6 +9,8 @@ import akka.actor.Props
import akka.pattern.ask
import testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec }
import akka.testkit._
import akka.actor.Identify
import akka.actor.ActorIdentity
object LookupRemoteActorMultiJvmSpec extends MultiNodeConfig {
@ -41,7 +43,10 @@ class LookupRemoteActorSpec extends MultiNodeSpec(LookupRemoteActorMultiJvmSpec)
"Remoting" must {
"lookup remote actor" taggedAs LongRunningTest in {
runOn(slave) {
val hello = system.actorFor(node(master) / "user" / "service-hello")
val hello = {
system.actorSelection(node(master) / "user" / "service-hello") ! Identify("id1")
expectMsgType[ActorIdentity].ref.get
}
hello.isInstanceOf[RemoteActorRef] must be(true)
val masterAddress = testConductor.getAddressFor(master).await
(hello ? "identify").await.asInstanceOf[ActorRef].path.address must equal(masterAddress)

View file

@ -4,7 +4,6 @@
package akka.remote.testconductor
import language.postfixOps
import com.typesafe.config.ConfigFactory
import akka.actor.Props
import akka.actor.Actor
@ -17,6 +16,8 @@ import java.net.InetSocketAddress
import java.net.InetAddress
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec, MultiNodeConfig }
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.actor.Identify
import akka.actor.ActorIdentity
object TestConductorMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false))
@ -36,7 +37,10 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with ST
def initialParticipants = 2
lazy val echo = system.actorFor(node(master) / "user" / "echo")
lazy val echo = {
system.actorSelection(node(master) / "user" / "echo") ! Identify(None)
expectMsgType[ActorIdentity].ref.get
}
"A TestConductor" must {

View file

@ -58,11 +58,7 @@ private[remote] class DefaultMessageDispatcher(private val system: ExtendedActor
if (UntrustedMode) log.debug("dropping daemon message in untrusted mode")
else {
if (LogReceive) log.debug("received daemon message {}", msgLog)
payload match {
case m @ (_: DaemonMsg | _: Terminated)
remoteDaemon ! m
case x log.debug("remoteDaemon received illegal message {} from {}", x, sender)
}
remoteDaemon ! payload
}
case l @ (_: LocalRef | _: RepointableRef) if l.isLocal

View file

@ -28,9 +28,8 @@ private[akka] object RemoteActorRefProvider {
case object WaitTransportShutdown extends TerminatorState
case object Finished extends TerminatorState
private class RemotingTerminator extends Actor with FSM[TerminatorState, Option[Internals]] {
private class RemotingTerminator(systemGuardian: ActorRef) extends Actor with FSM[TerminatorState, Option[Internals]] {
import context.dispatcher
val systemGuardian = context.actorFor("/system")
startWith(Uninitialized, None)
@ -79,10 +78,10 @@ private[akka] object RemoteActorRefProvider {
case _ super.!(message)(sender)
}
override def specialHandle(msg: Any): Boolean = msg match {
override def specialHandle(msg: Any, sender: ActorRef): Boolean = msg match {
// unwrap again in case the original message was DeadLetter(EndpointManager.Send(m))
case EndpointManager.Send(m, _, _) super.specialHandle(m)
case _ super.specialHandle(msg)
case EndpointManager.Send(m, _, _) super.specialHandle(m, sender)
case _ super.specialHandle(msg, sender)
}
@throws(classOf[java.io.ObjectStreamException])
@ -148,7 +147,7 @@ private[akka] class RemoteActorRefProvider(
def init(system: ActorSystemImpl): Unit = {
local.init(system)
remotingTerminator = system.systemActorOf(Props[RemotingTerminator], "remoting-terminator")
remotingTerminator = system.systemActorOf(Props(new RemotingTerminator(local.systemGuardian)), "remoting-terminator")
val internals = Internals(
remoteDaemon = {
@ -244,7 +243,7 @@ private[akka] class RemoteActorRefProvider(
new RemoteActorRef(transport, localAddress, rpath, supervisor, Some(props), Some(d))
} catch {
case NonFatal(e)
log.error(e, "Error while looking up address {}", addr)
log.error(e, "Error while looking up address [{}]", addr)
new EmptyLocalActorRef(this, path, eventStream)
}
}
@ -254,6 +253,7 @@ private[akka] class RemoteActorRefProvider(
}
}
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: ActorPath): InternalActorRef = {
if (hasAddress(path.address)) actorFor(rootGuardian, path.elements)
else try {
@ -261,11 +261,12 @@ private[akka] class RemoteActorRefProvider(
path, Nobody, props = None, deploy = None)
} catch {
case NonFatal(e)
log.error(e, "Error while looking up address {}", path.address)
log.error(e, "Error while looking up address [{}]", path.address)
new EmptyLocalActorRef(this, path, eventStream)
}
}
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
case ActorPathExtractor(address, elems)
if (hasAddress(address)) actorFor(rootGuardian, elems)
@ -276,40 +277,81 @@ private[akka] class RemoteActorRefProvider(
rootPath, Nobody, props = None, deploy = None)
} catch {
case NonFatal(e)
log.error(e, "Error while looking up address {}", rootPath.address)
log.error(e, "Error while looking up address [{}]", rootPath.address)
new EmptyLocalActorRef(this, rootPath, eventStream)
}
}
case _ local.actorFor(ref, path)
}
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef =
local.actorFor(ref, path)
def rootGuardianAt(address: Address): ActorRef =
if (hasAddress(address)) rootGuardian
else new RemoteActorRef(transport, transport.localAddressForRemote(address),
RootActorPath(address), Nobody, props = None, deploy = None)
/**
* INTERNAL API
* Called in deserialization of incoming remote messages. In this case the correct local address is known, therefore
* this method is faster than the actorFor above.
* Called in deserialization of incoming remote messages where the correct local address is known.
*/
def actorForWithLocalAddress(ref: InternalActorRef, path: String, localAddress: Address): InternalActorRef = {
private[akka] def resolveActorRefWithLocalAddress(path: String, localAddress: Address): InternalActorRef = {
path match {
case ActorPathExtractor(address, elems)
if (hasAddress(address)) actorFor(rootGuardian, elems)
else new RemoteActorRef(transport, localAddress,
new RootActorPath(address) / elems, Nobody, props = None, deploy = None)
if (hasAddress(address)) local.resolveActorRef(rootGuardian, elems)
else
new RemoteActorRef(transport, localAddress, RootActorPath(address) / elems,
Nobody, props = None, deploy = None)
case _
local.actorFor(ref, path)
log.debug("resolve of unknown path [{}] failed", path)
deadLetters
}
}
def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef =
local.actorFor(ref, path)
def resolveActorRef(path: String): ActorRef = path match {
case ActorPathExtractor(address, elems)
if (hasAddress(address)) local.resolveActorRef(rootGuardian, elems)
else {
val rootPath = RootActorPath(address) / elems
try {
new RemoteActorRef(transport, transport.localAddressForRemote(address),
rootPath, Nobody, props = None, deploy = None)
} catch {
case NonFatal(e)
log.error(e, "Error while resolving address [{}]", rootPath.address)
new EmptyLocalActorRef(this, rootPath, eventStream)
}
}
case _
log.debug("resolve of unknown path [{}] failed", path)
deadLetters
}
def resolveActorRef(path: ActorPath): ActorRef = {
if (hasAddress(path.address)) local.resolveActorRef(rootGuardian, path.elements)
else try {
new RemoteActorRef(transport, transport.localAddressForRemote(path.address),
path, Nobody, props = None, deploy = None)
} catch {
case NonFatal(e)
log.error(e, "Error while resolving address [{}]", path.address)
new EmptyLocalActorRef(this, path, eventStream)
}
}
/**
* Using (checking out) actor on a specific node.
*/
def useActorOnNode(path: ActorPath, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, path)
def useActorOnNode(ref: ActorRef, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, ref.path)
// we dont wait for the ACK, because the remote end will process this command before any other message to the new actor
actorFor(RootActorPath(path.address) / "remote") ! DaemonMsgCreate(props, deploy, path.toSerializationFormat, supervisor)
// actorSelection can't be used here because then it is not guaranteed that the actor is created
// before someone can send messages to it
resolveActorRef(RootActorPath(ref.path.address) / "remote") !
DaemonMsgCreate(props, deploy, ref.path.toSerializationFormat, supervisor)
}
def getExternalAddressFor(addr: Address): Option[Address] = {
@ -374,7 +416,7 @@ private[akka] class RemoteActorRef private[akka] (
override def provider: RemoteActorRefProvider = remote.provider
def start(): Unit =
if (props.isDefined && deploy.isDefined) remote.provider.useActorOnNode(path, props.get, deploy.get, getParent)
if (props.isDefined && deploy.isDefined) remote.provider.useActorOnNode(this, props.get, deploy.get, getParent)
def suspend(): Unit = sendSystemMessage(Suspend())

View file

@ -13,6 +13,11 @@ import akka.actor.ActorRefWithCell
import akka.actor.ActorRefScope
import akka.util.Switch
import akka.actor.RootActorPath
import akka.actor.SelectParent
import akka.actor.SelectChildName
import akka.actor.SelectChildPattern
import akka.actor.Identify
import akka.actor.ActorIdentity
/**
* INTERNAL API
@ -106,6 +111,21 @@ private[akka] class RemoteSystemDaemon(
}
}
case SelectParent(m) getParent.tell(m, sender)
case s @ SelectChildName(name, m)
getChild(s.allChildNames.iterator) match {
case Nobody
s.identifyRequest foreach { x sender ! ActorIdentity(x.messageId, None) }
case child
child.tell(s.wrappedMessage, sender)
}
case SelectChildPattern(p, m)
log.error("SelectChildPattern not allowed in actorSelection of remote deployed actors")
case Identify(messageId) sender ! ActorIdentity(messageId, Some(this))
case Terminated(child: ActorRefWithCell) if child.asInstanceOf[ActorRefScope].isLocal
terminating.locked {
removeChild(child.path.elements.drop(1).mkString("/"))

View file

@ -6,7 +6,7 @@ package akka.remote.serialization
import akka.serialization.{ Serializer, Serialization }
import com.google.protobuf.Message
import akka.actor.{ ActorSystem, ActorRef }
import akka.actor.{ ActorSystem, ActorRef, ExtendedActorSystem }
import akka.remote.RemoteProtocol.ActorRefProtocol
object ProtobufSerializer {
@ -24,8 +24,8 @@ object ProtobufSerializer {
* from Akka's protobuf representation in the supplied
* [[akka.actor.ActorSystem]].
*/
def deserializeActorRef(system: ActorSystem, refProtocol: ActorRefProtocol): ActorRef =
system.actorFor(refProtocol.getPath)
def deserializeActorRef(system: ExtendedActorSystem, refProtocol: ActorRefProtocol): ActorRef =
system.provider.resolveActorRef(refProtocol.getPath)
}
/**

View file

@ -140,6 +140,7 @@ abstract class ActorTransportAdapter(wrappedTransport: Transport, system: ActorS
// Write once variable initialized when Listen is called.
@volatile protected var manager: ActorRef = _
// FIXME #3074 how to replace actorFor here?
private def registerManager(): Future[ActorRef] =
(system.actorFor("/system/transports") ? RegisterTransportActor(managerProps, managerName)).mapTo[ActorRef]

View file

@ -144,11 +144,11 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
localAddress: Address): Message = {
val msgPdu = RemoteMessageProtocol.parseFrom(raw.toArray)
Message(
recipient = provider.actorForWithLocalAddress(provider.rootGuardian, msgPdu.getRecipient.getPath, localAddress),
recipient = provider.resolveActorRefWithLocalAddress(msgPdu.getRecipient.getPath, localAddress),
recipientAddress = AddressFromURIString(msgPdu.getRecipient.getPath),
serializedMessage = msgPdu.getMessage,
senderOption = if (!msgPdu.hasSender) None
else Some(provider.actorForWithLocalAddress(provider.rootGuardian, msgPdu.getSender.getPath, localAddress)))
else Some(provider.resolveActorRefWithLocalAddress(msgPdu.getSender.getPath, localAddress)))
}
private def decodeControlPdu(controlPdu: RemoteControlProtocol): AkkaPdu = {

View file

@ -49,4 +49,10 @@ akka {
expectMsg(60.seconds, path)
}
"receive ActorIdentity(None) when identified node is unknown host" in {
val path = RootActorPath(Address("akka.tcp", system.name, "unknownhost", 2552)) / "user" / "subject"
system.actorSelection(path) ! Identify(path)
expectMsg(60.seconds, ActorIdentity(path, None))
}
}

View file

@ -15,13 +15,18 @@ import scala.concurrent.Future
import scala.concurrent.duration._
object RemotingSpec {
case class ActorForReq(s: String)
case class ActorSelReq(s: String)
class Echo1 extends Actor {
var target: ActorRef = context.system.deadLetters
def receive = {
case (p: Props, n: String) sender ! context.actorOf(Props[Echo1], n)
case ex: Exception throw ex
case s: String sender ! context.actorFor(s)
case ActorForReq(s) sender ! context.actorFor(s)
case ActorSelReq(s) sender ! context.actorSelection(s)
case x target = sender; sender ! x
}
@ -110,8 +115,10 @@ object RemotingSpec {
actor.deployment {
/blub.remote = "akka.test://remote-sys@localhost:12346"
/looker/child.remote = "akka.test://remote-sys@localhost:12346"
/looker/child/grandchild.remote = "akka.test://RemotingSpec@localhost:12345"
/looker1/child.remote = "akka.test://remote-sys@localhost:12346"
/looker1/child/grandchild.remote = "akka.test://RemotingSpec@localhost:12345"
/looker2/child.remote = "akka.test://remote-sys@localhost:12346"
/looker2/child/grandchild.remote = "akka.test://RemotingSpec@localhost:12345"
}
}
""")
@ -234,8 +241,8 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
EventFilter.warning(pattern = "received dead letter.*")))
sys.actorOf(Props[Echo2], name = "echo")
}
val moreRefs = moreSystems map (sys system.actorFor(RootActorPath(addr(sys, "tcp")) / "user" / "echo"))
val aliveEcho = system.actorFor(RootActorPath(addr(remoteSystem, "tcp")) / "user" / "echo")
val moreRefs = moreSystems map (sys system.actorSelection(RootActorPath(addr(sys, "tcp")) / "user" / "echo"))
val aliveEcho = system.actorSelection(RootActorPath(addr(remoteSystem, "tcp")) / "user" / "echo")
val n = 100
// first everything is up and running
@ -302,15 +309,21 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
system.actorFor("akka.test://remote-sys@localhost:12346/user/otherEcho1") ! 76
expectMsg(76)
remoteSystem.actorSelection("/user/otherEcho1") ! 77
expectMsg(77)
system.actorSelection("akka.test://remote-sys@localhost:12346/user/otherEcho1") ! 78
expectMsg(78)
}
"look-up actors across node boundaries" in {
val l = system.actorOf(Props(new Actor {
def receive = {
case (p: Props, n: String) sender ! context.actorOf(p, n)
case s: String sender ! context.actorFor(s)
case ActorForReq(s) sender ! context.actorFor(s)
}
}), "looker")
}), "looker1")
// child is configured to be deployed on remote-sys (remoteSystem)
l ! ((Props[Echo1], "child"))
val child = expectMsgType[ActorRef]
@ -320,16 +333,16 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
grandchild.asInstanceOf[ActorRefScope].isLocal must be(true)
grandchild ! 43
expectMsg(43)
val myref = system.actorFor(system / "looker" / "child" / "grandchild")
val myref = system.actorFor(system / "looker1" / "child" / "grandchild")
myref.isInstanceOf[RemoteActorRef] must be(true)
myref ! 44
expectMsg(44)
lastSender must be(grandchild)
lastSender must be theSameInstanceAs grandchild
child.asInstanceOf[RemoteActorRef].getParent must be(l)
system.actorFor("/user/looker/child") must be theSameInstanceAs child
Await.result(l ? "child/..", timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
Await.result(system.actorFor(system / "looker" / "child") ? "..", timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
system.actorFor("/user/looker1/child") must be theSameInstanceAs child
Await.result(l ? ActorForReq("child/.."), timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
Await.result(system.actorFor(system / "looker1" / "child") ? ActorForReq(".."), timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
watch(child)
child ! PoisonPill
@ -343,9 +356,66 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
child2.path.uid must not be (child.path.uid)
child ! 46
expectNoMsg(1.second)
system.actorFor(system / "looker" / "child") ! 47
system.actorFor(system / "looker1" / "child") ! 47
expectMsg(47)
}
"select actors across node boundaries" in {
val l = system.actorOf(Props(new Actor {
def receive = {
case (p: Props, n: String) sender ! context.actorOf(p, n)
case ActorSelReq(s) sender ! context.actorSelection(s)
}
}), "looker2")
// child is configured to be deployed on remoteSystem
l ! (Props[Echo1], "child")
val child = expectMsgType[ActorRef]
// grandchild is configured to be deployed on RemotingSpec (system)
child ! (Props[Echo1], "grandchild")
val grandchild = expectMsgType[ActorRef]
grandchild.asInstanceOf[ActorRefScope].isLocal must be(true)
grandchild ! 53
expectMsg(53)
val mysel = system.actorSelection(system / "looker2" / "child" / "grandchild")
mysel ! 54
expectMsg(54)
lastSender must be(grandchild)
lastSender must be theSameInstanceAs grandchild
mysel ! Identify(mysel)
val grandchild2 = expectMsgType[ActorIdentity].ref
grandchild2 must be === Some(grandchild)
system.actorSelection("/user/looker2/child") ! Identify(None)
expectMsgType[ActorIdentity].ref must be === Some(child)
l ! ActorSelReq("child/..")
expectMsgType[ActorSelection] ! Identify(None)
expectMsgType[ActorIdentity].ref.get must be theSameInstanceAs l
system.actorSelection(system / "looker2" / "child") ! ActorSelReq("..")
expectMsgType[ActorSelection] ! Identify(None)
expectMsgType[ActorIdentity].ref.get must be theSameInstanceAs l
child ! Identify("idReq1")
expectMsg(ActorIdentity("idReq1", Some(child)))
watch(child)
child ! PoisonPill
expectMsg("postStop")
expectMsgType[Terminated].actor must be === child
l ! (Props[Echo1], "child")
val child2 = expectMsgType[ActorRef]
child2 ! Identify("idReq2")
expectMsg(ActorIdentity("idReq2", Some(child2)))
system.actorSelection(child.path) ! Identify("idReq3")
expectMsg(ActorIdentity("idReq3", Some(child2)))
child ! Identify("idReq4")
expectMsg(ActorIdentity("idReq4", None))
child2 ! 55
expectMsg(55)
// msg to old ActorRef (different uid) should not get through
child2.path.uid must not be (child.path.uid)
child ! 56
expectNoMsg(1.second)
system.actorSelection(system / "looker2" / "child") ! 57
expectMsg(57)
}
"not fail ask across node boundaries" in {
@ -455,8 +525,10 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
val otherGuyRemoteTest = otherGuy.path.toSerializationFormatWithAddress(addr(otherSystem, "test"))
val remoteEchoHereSsl = system.actorFor(s"akka.ssl.tcp://remote-sys@localhost:${port(remoteSystem, "ssl.tcp")}/user/echo")
val proxySsl = system.actorOf(Props(new Proxy(remoteEchoHereSsl, testActor)), "proxy-ssl")
EventFilter[RemoteTransportException](start = "Error while resolving address", occurrences = 1).intercept {
proxySsl ! otherGuy
expectMsg(3.seconds, ("pong", otherGuyRemoteTest))
}(otherSystem)
} finally {
otherSystem.shutdown()
otherSystem.awaitTermination(5.seconds.dilated)

View file

@ -139,7 +139,10 @@ abstract class Ticket1978CommunicationSpec(val cipherConfig: CipherConfig) exten
val otherAddress = other.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.defaultAddress
"support tell" in {
val here = system.actorFor(otherAddress.toString + "/user/echo")
val here = {
system.actorSelection(otherAddress.toString + "/user/echo") ! Identify(None)
expectMsgType[ActorIdentity].ref.get
}
for (i 1 to 1000) here ! (("ping", i))
for (i 1 to 1000) expectMsgPF(timeout.duration) { case (("pong", i), `testActor`) true }
@ -147,7 +150,10 @@ abstract class Ticket1978CommunicationSpec(val cipherConfig: CipherConfig) exten
"support ask" in {
import system.dispatcher
val here = system.actorFor(otherAddress.toString + "/user/echo")
val here = {
system.actorSelection(otherAddress.toString + "/user/echo") ! Identify(None)
expectMsgType[ActorIdentity].ref.get
}
val f = for (i 1 to 1000) yield here ? (("ping", i)) mapTo classTag[((String, Int), ActorRef)]
Await.result(Future.sequence(f), timeout.duration).map(_._1._1).toSet must be(Set("pong"))

View file

@ -75,7 +75,11 @@ class AkkaProtocolStressTest extends AkkaSpec(configA) with ImplicitSender with
val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
val rootB = RootActorPath(addressB)
val here = system.actorFor(rootB / "user" / "echo")
val here = {
val path =
system.actorSelection(rootB / "user" / "echo") ! Identify(None)
expectMsgType[ActorIdentity].ref.get
}
"AkkaProtocolTransport" must {
"guarantee at-most-once delivery and message ordering despite packet loss" taggedAs TimingTest in {

View file

@ -66,7 +66,10 @@ class ThrottlerTransportAdapterSpec extends AkkaSpec(configA) with ImplicitSende
val remote = systemB.actorOf(Props[Echo], "echo")
val rootB = RootActorPath(systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress)
val here = system.actorFor(rootB / "user" / "echo")
val here = {
system.actorSelection(rootB / "user" / "echo") ! Identify(None)
expectMsgType[ActorIdentity].ref.get
}
def throttle(direction: Direction, mode: ThrottleMode): Boolean = {
val rootBAddress = Address("akka", "systemB", "localhost", rootB.address.port.get)

View file

@ -1,21 +1,16 @@
package sample.cluster.stats.japi;
import scala.concurrent.Future;
import sample.cluster.stats.japi.StatsMessages.JobFailed;
import sample.cluster.stats.japi.StatsMessages.StatsJob;
import akka.actor.ActorRef;
import akka.actor.ActorSelection;
import akka.actor.Address;
import akka.actor.UntypedActor;
import akka.dispatch.Recover;
import akka.cluster.Cluster;
import akka.cluster.ClusterEvent.CurrentClusterState;
import akka.cluster.ClusterEvent.RoleLeaderChanged;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.util.Timeout;
import static akka.pattern.Patterns.ask;
import static akka.pattern.Patterns.pipe;
import static java.util.concurrent.TimeUnit.SECONDS;
//#facade
public class StatsFacade extends UntypedActor {
@ -23,7 +18,7 @@ public class StatsFacade extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
Cluster cluster = Cluster.get(getContext().system());
Address currentMaster = null;
ActorSelection currentMaster = null;
//subscribe to cluster changes, RoleLeaderChanged
@Override
@ -44,30 +39,29 @@ public class StatsFacade extends UntypedActor {
getSelf());
} else if (message instanceof StatsJob) {
StatsJob job = (StatsJob) message;
ActorRef service = getContext().actorFor(currentMaster +
"/user/singleton/statsService");
Future<Object> f = ask(service, job, new Timeout(5, SECONDS)).recover(
new Recover<Object>() {
public Object recover(Throwable t) {
return new JobFailed("Service unavailable, try again later");
}
}, getContext().dispatcher());
pipe(f, getContext().dispatcher()).to(getSender());
currentMaster.tell(message, getSender());
} else if (message instanceof CurrentClusterState) {
CurrentClusterState state = (CurrentClusterState) message;
currentMaster = state.getRoleLeader("compute");
setCurrentMaster(state.getRoleLeader("compute"));
} else if (message instanceof RoleLeaderChanged) {
RoleLeaderChanged leaderChanged = (RoleLeaderChanged) message;
if (leaderChanged.role().equals("compute"))
currentMaster = leaderChanged.getLeader();
setCurrentMaster(leaderChanged.getLeader());
} else {
unhandled(message);
}
}
void setCurrentMaster(Address address) {
if (address == null)
currentMaster = null;
else
currentMaster = getContext().actorSelection(address +
"/user/singleton/statsService");
}
}
//#facade

View file

@ -12,7 +12,7 @@ import sample.cluster.stats.japi.StatsMessages.StatsResult;
import scala.concurrent.forkjoin.ThreadLocalRandom;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import akka.actor.ActorRef;
import akka.actor.ActorSelection;
import akka.actor.Address;
import akka.actor.Cancellable;
import akka.actor.UntypedActor;
@ -61,7 +61,7 @@ public class StatsSampleClient extends UntypedActor {
List<Address> nodesList = new ArrayList<Address>(nodes);
Address address = nodesList.get(ThreadLocalRandom.current().nextInt(
nodesList.size()));
ActorRef service = getContext().actorFor(address + servicePath);
ActorSelection service = getContext().actorSelection(address + servicePath);
service.tell(new StatsJob("this is the text that will be analyzed"),
getSelf());

View file

@ -56,7 +56,7 @@ public class TransformationBackend extends UntypedActor {
void register(Member member) {
if (member.hasRole("frontend"))
getContext().actorFor(member.address() + "/user/frontend").tell(
getContext().actorSelection(member.address() + "/user/frontend").tell(
BACKEND_REGISTRATION, getSelf());
}
}

View file

@ -8,6 +8,7 @@ import com.typesafe.config.ConfigFactory
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.ActorSelection
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.PoisonPill
@ -21,9 +22,6 @@ import akka.cluster.MemberStatus
import akka.contrib.pattern.ClusterSingletonManager
import akka.routing.FromConfig
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
import akka.pattern.ask
import akka.pattern.pipe
import akka.util.Timeout
//#imports
//#messages
@ -93,7 +91,7 @@ class StatsFacade extends Actor with ActorLogging {
import context.dispatcher
val cluster = Cluster(context.system)
var currentMaster: Option[Address] = None
var currentMaster: Option[ActorSelection] = None
// subscribe to cluster changes, RoleLeaderChanged
// re-subscribe when restart
@ -104,19 +102,17 @@ class StatsFacade extends Actor with ActorLogging {
case job: StatsJob if currentMaster.isEmpty
sender ! JobFailed("Service unavailable, try again later")
case job: StatsJob
implicit val timeout = Timeout(5.seconds)
currentMaster foreach { address
val service = context.actorFor(RootActorPath(address) /
"user" / "singleton" / "statsService")
service ? job recover {
case _ JobFailed("Service unavailable, try again later")
} pipeTo sender
}
currentMaster foreach { _.tell(job, sender) }
case state: CurrentClusterState
currentMaster = state.roleLeader("compute")
setCurrentMaster(state.roleLeader("compute"))
case RoleLeaderChanged(role, leader)
if (role == "compute")
currentMaster = leader
setCurrentMaster(leader)
}
def setCurrentMaster(address: Option[Address]): Unit = {
currentMaster = address.map(a context.actorSelection(RootActorPath(a) /
"user" / "singleton" / "statsService"))
}
}
@ -200,7 +196,7 @@ class StatsSampleClient(servicePath: String) extends Actor {
case "tick" if nodes.nonEmpty
// just pick any one
val address = nodes.toIndexedSeq(ThreadLocalRandom.current.nextInt(nodes.size))
val service = context.actorFor(RootActorPath(address) / servicePathElements)
val service = context.actorSelection(RootActorPath(address) / servicePathElements)
service ! StatsJob("this is the text that will be analyzed")
case result: StatsResult
println(result)

View file

@ -109,7 +109,7 @@ class TransformationBackend extends Actor {
def register(member: Member): Unit =
if (member.hasRole("frontend"))
context.actorFor(RootActorPath(member.address) / "user" / "frontend") !
context.actorSelection(RootActorPath(member.address) / "user" / "frontend") !
BackendRegistration
}
//#backend

View file

@ -93,18 +93,13 @@ abstract class StatsSampleSingleMasterSpec extends MultiNodeSpec(StatsSampleSing
}
"show usage of the statsFacade" in within(40 seconds) {
val facade = system.actorFor(RootActorPath(node(third).address) / "user" / "statsFacade")
val facade = system.actorSelection(RootActorPath(node(third).address) / "user" / "statsFacade")
// eventually the service should be ok,
// service and worker nodes might not be up yet
awaitCond {
awaitAssert {
facade ! StatsJob("this is the text that will be analyzed")
expectMsgPF() {
case unavailble: JobFailed false
case StatsResult(meanWordLength)
meanWordLength must be(3.875 plusOrMinus 0.001)
true
}
expectMsgType[StatsResult](1.second).meanWordLength must be(3.875 plusOrMinus 0.001)
}
testConductor.enter("done")

View file

@ -118,17 +118,12 @@ abstract class StatsSampleSpec extends MultiNodeSpec(StatsSampleSpecConfig)
}
def assertServiceOk(): Unit = {
val service = system.actorFor(node(third) / "user" / "statsService")
val service = system.actorSelection(node(third) / "user" / "statsService")
// eventually the service should be ok,
// first attempts might fail because worker actors not started yet
awaitCond {
awaitAssert {
service ! StatsJob("this is the text that will be analyzed")
expectMsgPF() {
case unavailble: JobFailed false
case StatsResult(meanWordLength)
meanWordLength must be(3.875 plusOrMinus 0.001)
true
}
expectMsgType[StatsResult](1.second).meanWordLength must be(3.875 plusOrMinus 0.001)
}
}

View file

@ -101,17 +101,12 @@ abstract class StatsSampleJapiSpec extends MultiNodeSpec(StatsSampleJapiSpecConf
}
def assertServiceOk(): Unit = {
val service = system.actorFor(node(third) / "user" / "statsService")
val service = system.actorSelection(node(third) / "user" / "statsService")
// eventually the service should be ok,
// first attempts might fail because worker actors not started yet
awaitCond {
awaitAssert {
service ! new StatsJob("this is the text that will be analyzed")
expectMsgPF() {
case unavailble: JobFailed false
case r: StatsResult
r.getMeanWordLength must be(3.875 plusOrMinus 0.001)
true
}
expectMsgType[StatsResult](1.second).getMeanWordLength must be(3.875 plusOrMinus 0.001)
}
}
//#test-statsService

View file

@ -97,18 +97,13 @@ abstract class StatsSampleSingleMasterJapiSpec extends MultiNodeSpec(StatsSample
}
"show usage of the statsFacade" in within(40 seconds) {
val facade = system.actorFor(RootActorPath(node(third).address) / "user" / "statsFacade")
val facade = system.actorSelection(RootActorPath(node(third).address) / "user" / "statsFacade")
// eventually the service should be ok,
// service and worker nodes might not be up yet
awaitCond {
awaitAssert {
facade ! new StatsJob("this is the text that will be analyzed")
expectMsgPF() {
case unavailble: JobFailed false
case r: StatsResult
r.getMeanWordLength must be(3.875 plusOrMinus 0.001)
true
}
expectMsgType[StatsResult](1.second).getMeanWordLength must be(3.875 plusOrMinus 0.001)
}
testConductor.enter("done")

View file

@ -115,17 +115,12 @@ abstract class TransformationSampleSpec extends MultiNodeSpec(TransformationSamp
}
def assertServiceOk(): Unit = {
val transformationFrontend = system.actorFor("akka://" + system.name + "/user/frontend")
val transformationFrontend = system.actorSelection("akka://" + system.name + "/user/frontend")
// eventually the service should be ok,
// backends might not have registered initially
awaitCond {
awaitAssert {
transformationFrontend ! TransformationJob("hello")
expectMsgPF() {
case unavailble: JobFailed false
case TransformationResult(result)
result must be("HELLO")
true
}
expectMsgType[TransformationResult](1.second).text must be("HELLO")
}
}

View file

@ -116,17 +116,12 @@ abstract class TransformationSampleJapiSpec extends MultiNodeSpec(Transformation
}
def assertServiceOk(): Unit = {
val transformationFrontend = system.actorFor("akka://" + system.name + "/user/frontend")
val transformationFrontend = system.actorSelection("akka://" + system.name + "/user/frontend")
// eventually the service should be ok,
// backends might not have registered initially
awaitCond {
awaitAssert {
transformationFrontend ! new TransformationJob("hello")
expectMsgPF() {
case unavailble: JobFailed false
case r: TransformationResult
r.getText must be("HELLO")
true
}
expectMsgType[TransformationResult](1.second).getText must be("HELLO")
}
}

View file

@ -96,6 +96,8 @@ Now you should see something like this::
[INFO] [01/25/2013 15:05:53.954] [run-main] [Remoting] Starting remoting
[INFO] [01/25/2013 15:05:54.769] [run-main] [Remoting] Remoting started; listening on addresses :[akka.tcp://LookupApplication@127.0.0.1:2553]
Started Lookup Application
Not ready yet
Not ready yet
Add result: 0 + 22 = 22
Add result: 41 + 71 = 112
Add result: 61 + 14 = 75

View file

@ -1,26 +0,0 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.remote.calculator.java;
import akka.actor.ActorRef;
public class InternalMsg {
static class MathOpMsg {
private final ActorRef actor;
private final Op.MathOp mathOp;
MathOpMsg(ActorRef actor, Op.MathOp mathOp) {
this.actor = actor;
this.mathOp = mathOp;
}
public ActorRef getActor() {
return actor;
}
public Op.MathOp getMathOp() {
return mathOp;
}
}
}

View file

@ -3,37 +3,35 @@
*/
package sample.remote.calculator.java;
import akka.actor.ActorRef;
import akka.actor.UntypedActor;
import java.text.DecimalFormat;
import java.text.NumberFormat;
//#actor
public class JCreationActor extends UntypedActor {
private static final NumberFormat formatter = new DecimalFormat("#0.00");
private final ActorRef remoteActor;
public JCreationActor(ActorRef remoteActor) {
this.remoteActor = remoteActor;
}
@Override
public void onReceive(Object message) throws Exception {
if (message instanceof InternalMsg.MathOpMsg) {
// forward math op to server actor
InternalMsg.MathOpMsg msg = (InternalMsg.MathOpMsg) message;
msg.getActor().tell(msg.getMathOp(), getSelf());
if (message instanceof Op.MathOp) {
// send message to server actor
remoteActor.tell(message, getSelf());
} else if (message instanceof Op.MathResult) {
// receive reply from server actor
if (message instanceof Op.MultiplicationResult) {
} else if (message instanceof Op.MultiplicationResult) {
Op.MultiplicationResult result = (Op.MultiplicationResult) message;
System.out.println("Mul result: " + result.getN1() + " * " +
result.getN2() + " = " + result.getResult());
System.out.printf("Mul result: %d * %d = %d\n",
result.getN1(), result.getN2(), result.getResult());
} else if (message instanceof Op.DivisionResult) {
Op.DivisionResult result = (Op.DivisionResult) message;
System.out.println("Div result: " + result.getN1() + " / " +
result.getN2() + " = " + formatter.format(result.getResult()));
}
System.out.printf("Div result: %.0f / %d = %.2f\n",
result.getN1(), result.getN2(), result.getResult());
} else {
unhandled(message);
}

View file

@ -6,6 +6,8 @@ package sample.remote.calculator.java;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.kernel.Bootable;
import com.typesafe.config.ConfigFactory;
@ -13,18 +15,22 @@ import com.typesafe.config.ConfigFactory;
public class JCreationApplication implements Bootable {
private ActorSystem system;
private ActorRef actor;
private ActorRef remoteActor;
public JCreationApplication() {
system = ActorSystem.create("CreationApplication", ConfigFactory.load()
.getConfig("remotecreation"));
actor = system.actorOf(new Props(JCreationActor.class));
remoteActor = system.actorOf(new Props(JAdvancedCalculatorActor.class),
final ActorRef remoteActor = system.actorOf(new Props(JAdvancedCalculatorActor.class),
"advancedCalculator");
actor = system.actorOf(new Props().withCreator(new UntypedActorFactory() {
public UntypedActor create() {
return new JCreationActor(remoteActor);
}
}), "creationActor");
}
public void doSomething(Op.MathOp mathOp) {
actor.tell(new InternalMsg.MathOpMsg(remoteActor, mathOp), null);
actor.tell(mathOp, null);
}
@Override

View file

@ -3,34 +3,51 @@
*/
package sample.remote.calculator.java;
import akka.actor.ActorRef;
import akka.actor.ActorIdentity;
import akka.actor.Identify;
import akka.actor.UntypedActor;
import akka.actor.ReceiveTimeout;
//#actor
public class JLookupActor extends UntypedActor {
private final String path;
private ActorRef remoteActor = null;
public JLookupActor(String path) {
this.path = path;
sendIdentifyRequest();
}
private void sendIdentifyRequest() {
getContext().actorSelection(path).tell(new Identify(path), getSelf());
}
@Override
public void onReceive(Object message) throws Exception {
if (message instanceof InternalMsg.MathOpMsg) {
if (message instanceof ActorIdentity) {
remoteActor = ((ActorIdentity) message).getRef();
} else if (message.equals(ReceiveTimeout.getInstance())) {
sendIdentifyRequest();
} else if (remoteActor == null) {
System.out.println("Not ready yet");
} else if (message instanceof Op.MathOp) {
// send message to server actor
InternalMsg.MathOpMsg msg = (InternalMsg.MathOpMsg) message;
msg.getActor().tell(msg.getMathOp(), getSelf());
remoteActor.tell(message, getSelf());
} else if (message instanceof Op.MathResult) {
// receive reply from server actor
if (message instanceof Op.AddResult) {
} else if (message instanceof Op.AddResult) {
Op.AddResult result = (Op.AddResult) message;
System.out.println("Add result: " + result.getN1() + " + " +
result.getN2() + " = " + result.getResult());
System.out.printf("Add result: %d + %d = %d\n", result.getN1(), result.getN2(), result.getResult());
} else if (message instanceof Op.SubtractResult) {
Op.SubtractResult result = (Op.SubtractResult) message;
System.out.println("Sub result: " + result.getN1() + " - " +
result.getN2() + " = " + result.getResult());
}
System.out.printf("Sub result: %d - %d = %d\n", result.getN1(), result.getN2(), result.getResult());
} else {
unhandled(message);
}

View file

@ -7,6 +7,8 @@ package sample.remote.calculator.java;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.kernel.Bootable;
import com.typesafe.config.ConfigFactory;
//#imports
@ -15,18 +17,19 @@ import com.typesafe.config.ConfigFactory;
public class JLookupApplication implements Bootable {
private ActorSystem system;
private ActorRef actor;
private ActorRef remoteActor;
public JLookupApplication() {
system = ActorSystem.create("LookupApplication", ConfigFactory.load()
.getConfig("remotelookup"));
actor = system.actorOf(new Props(JLookupActor.class));
remoteActor = system.actorFor(
"akka.tcp://CalculatorApplication@127.0.0.1:2552/user/simpleCalculator");
system = ActorSystem.create("LookupApplication", ConfigFactory.load().getConfig("remotelookup"));
final String path = "akka.tcp://CalculatorApplication@127.0.0.1:2552/user/simpleCalculator";
actor = system.actorOf(new Props().withCreator(new UntypedActorFactory() {
public UntypedActor create() {
return new JLookupActor(path);
}
}), "lookupActor");
}
public void doSomething(Op.MathOp mathOp) {
actor.tell(new InternalMsg.MathOpMsg(remoteActor, mathOp), null);
actor.tell(mathOp, null);
}
@Override

View file

@ -13,18 +13,16 @@ public class JSimpleCalculatorActor extends UntypedActor {
if (message instanceof Op.Add) {
Op.Add add = (Op.Add) message;
System.out.println("Calculating " + add.getN1() + " + " + add.getN2());
getSender()
.tell(
new Op.AddResult(add.getN1(), add.getN2(), add.getN1()
+ add.getN2()), getSelf());
getSender().tell(new Op.AddResult(
add.getN1(), add.getN2(), add.getN1() + add.getN2()),
getSelf());
} else if (message instanceof Op.Subtract) {
Op.Subtract subtract = (Op.Subtract) message;
System.out.println("Calculating " + subtract.getN1() + " - "
+ subtract.getN2());
getSender().tell(
new Op.SubtractResult(subtract.getN1(), subtract.getN2(),
subtract.getN1() - subtract.getN2()), getSelf());
System.out.println("Calculating " + subtract.getN1() + " - " + subtract.getN2());
getSender().tell(new Op.SubtractResult(
subtract.getN1(), subtract.getN2(), subtract.getN1() - subtract.getN2()),
getSelf());
} else {
unhandled(message);

View file

@ -16,12 +16,13 @@ class CreationApplication extends Bootable {
//#setup
val system =
ActorSystem("RemoteCreation", ConfigFactory.load.getConfig("remotecreation"))
val localActor = system.actorOf(Props[CreationActor], "creationActor")
val remoteActor =
system.actorOf(Props[AdvancedCalculatorActor], "advancedCalculator")
val remoteActor = system.actorOf(Props[AdvancedCalculatorActor],
name = "advancedCalculator")
val localActor = system.actorOf(Props(new CreationActor(remoteActor)),
name = "creationActor")
def doSomething(op: MathOp): Unit =
localActor ! ((remoteActor, op))
localActor ! op
//#setup
def startup() {
@ -33,14 +34,14 @@ class CreationApplication extends Bootable {
}
//#actor
class CreationActor extends Actor {
class CreationActor(remoteActor: ActorRef) extends Actor {
def receive = {
case (actor: ActorRef, op: MathOp) actor ! op
case op: MathOp remoteActor ! op
case result: MathResult result match {
case MultiplicationResult(n1, n2, r)
println("Mul result: %d * %d = %d".format(n1, n2, r))
printf("Mul result: %d * %d = %d\n", n1, n2, r)
case DivisionResult(n1, n2, r)
println("Div result: %.0f / %d = %.2f".format(n1, n2, r))
printf("Div result: %.0f / %d = %.2f\n", n1, n2, r)
}
}
}

View file

@ -7,23 +7,26 @@ package sample.remote.calculator
* comments like //#<tag> are there for inclusion into docs, please dont remove
*/
import akka.kernel.Bootable
import scala.util.Random
//#imports
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.{ ActorRef, Props, Actor, ActorSystem }
import akka.actor.Identify
import akka.actor.ActorIdentity
import akka.kernel.Bootable
import akka.actor.ReceiveTimeout
//#imports
class LookupApplication extends Bootable {
//#setup
val system =
ActorSystem("LookupApplication", ConfigFactory.load.getConfig("remotelookup"))
val actor = system.actorOf(Props[LookupActor], "lookupActor")
val remoteActor = system.actorFor(
"akka.tcp://CalculatorApplication@127.0.0.1:2552/user/simpleCalculator")
val remotePath =
"akka.tcp://CalculatorApplication@127.0.0.1:2552/user/simpleCalculator"
val actor = system.actorOf(Props(new LookupActor(remotePath)), "lookupActor")
def doSomething(op: MathOp): Unit =
actor ! ((remoteActor, op))
actor ! op
//#setup
def startup() {
@ -35,14 +38,30 @@ class LookupApplication extends Bootable {
}
//#actor
class LookupActor extends Actor {
class LookupActor(path: String) extends Actor {
context.setReceiveTimeout(3.seconds)
sendIdentifyRequest()
def sendIdentifyRequest(): Unit =
context.actorSelection(path) ! Identify(path)
def receive = {
case (actor: ActorRef, op: MathOp) actor ! op
case ActorIdentity(`path`, Some(actor))
context.setReceiveTimeout(Duration.Undefined)
context.become(active(actor))
case ActorIdentity(`path`, None) println(s"Remote actor not availible: $path")
case ReceiveTimeout sendIdentifyRequest()
case _ println("Not ready yet")
}
def active(actor: ActorRef): Actor.Receive = {
case op: MathOp actor ! op
case result: MathResult result match {
case AddResult(n1, n2, r)
println("Add result: %d + %d = %d".format(n1, n2, r))
printf("Add result: %d + %d = %d\n", n1, n2, r)
case SubtractResult(n1, n2, r)
println("Sub result: %d - %d = %d".format(n1, n2, r))
printf("Sub result: %d - %d = %d\n", n1, n2, r)
}
}
}

View file

@ -53,7 +53,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
val latch = new TestLatch(1)(system)
system.registerOnTermination(latch.countDown())
system.actorFor("/") ! PoisonPill
system.actorSelection("/") ! PoisonPill
Await.ready(latch, 2 seconds)
}

View file

@ -87,7 +87,11 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout {
"watch actors when queue non-empty" in {
val probe = TestProbe()
val target = system.actorFor("/nonexistent") // deadLetters does not send Terminated
// deadLetters does not send Terminated
val target = system.actorOf(Props(new Actor {
def receive = Actor.emptyBehavior
}))
system.stop(target)
probe.ref ! "hello"
probe watch target
probe.expectMsg(1.seconds, "hello")