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.duration._
import scala.concurrent.Await import scala.concurrent.Await
import akka.pattern.ask import akka.pattern.ask
import java.net.MalformedURLException
object ActorLookupSpec { 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.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import java.net.MalformedURLException
class ActorPathSpec extends WordSpec with MustMatchers { 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 { "create correct toString" in {
val a = Address("akka.tcp", "mysys") 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") (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 { "create correct toStringWithAddress" in {
val local = Address("akka.tcp", "mysys") val local = Address("akka.tcp", "mysys")
val a = local.copy(host = Some("aaa"), port = Some(2552)) 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 { "shut down when /user fails" in {
implicit val system = ActorSystem("Stop", AkkaSpec.testConf) implicit val system = ActorSystem("Stop", AkkaSpec.testConf)
EventFilter[ActorKilledException]() intercept { EventFilter[ActorKilledException]() intercept {
system.actorFor("/user") ! Kill system.actorSelection("/user") ! Kill
awaitCond(system.isTerminated) awaitCond(system.isTerminated)
} }
} }

View file

@ -5,10 +5,11 @@
package akka.actor package akka.actor
import akka.AkkaException import akka.AkkaException
import scala.collection.immutable
import scala.annotation.tailrec
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import java.util.regex.Pattern import java.util.regex.Pattern
import scala.annotation.tailrec
/** /**
* INTERNAL API * INTERNAL API
@ -54,6 +55,30 @@ case object Kill extends Kill {
def getInstance = this 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) * When Death Watch is used, the watcher will receive a Terminated(watched)
* message when watched is terminated. * 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 * ActorRefFactory.actorSelection returns a special ref which sends these
* nested path descriptions whenever using ! on them, the idea being that the * nested path descriptions whenever using ! on them, the idea being that the
* message is delivered by active routing of the various actors involved. * 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 * INTERNAL API
*/ */
@SerialVersionUID(1L) @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 * 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 * Returns all supervised children; this method returns a view (i.e. a lazy
* collection) onto the internal collection of children. Targeted lookups * 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") * val badLookup = context.children find (_.path.name == "kid")
* // should better be expressed as: * // should better be expressed as:
* val goodLookup = context.actorFor("kid") * val goodLookup = context.child("kid")
* }}} * }}}
*/ */
def children: immutable.Iterable[ActorRef] def children: immutable.Iterable[ActorRef]
@ -475,9 +475,27 @@ private[akka] class ActorCell(
case AddressTerminated(address) addressTerminated(address) case AddressTerminated(address) addressTerminated(address)
case Kill throw new ActorKilledException("Kill") case Kill throw new ActorKilledException("Kill")
case PoisonPill self.stop() case PoisonPill self.stop()
case SelectParent(m) parent.tell(m, msg.sender) case SelectParent(m)
case SelectChildName(name, m) getChildByName(name) match { case Some(c: ChildRestartStats) c.child.tell(m, msg.sender); case _ } if (self == system.provider.rootGuardian) self.tell(m, msg.sender)
case SelectChildPattern(p, m) for (c children if p.matcher(c.path.name).matches) c.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 = override private[akka] def withUid(uid: Int): ActorPath =
if (uid == ActorCell.undefinedUid) this 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). * there is nobody to reply to).
* *
* <pre> * <pre>
* actor.tell(message, context); * actor.tell(message, getSelf());
* </pre> * </pre>
*/ */
final def tell(msg: Any, sender: ActorRef): Unit = this.!(msg)(sender) 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 * Method for looking up a single child beneath this actor. Override in order
* to inject synthetic actor paths like /temp. * 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) val (childName, uid) = ActorCell.splitNameAndUid(name)
actorCell.getChildByName(childName) match { actorCell.getChildByName(childName) match {
case Some(crs: ChildRestartStats) if uid == ActorCell.undefinedUid || uid == crs.uid 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." + "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use 'akka.serialization.Serialization.currentSystem.withValue(system) { ... }'") " Use 'akka.serialization.Serialization.currentSystem.withValue(system) { ... }'")
case someSystem case someSystem
someSystem.actorFor(path) someSystem.provider.resolveActorRef(path)
} }
} }
@ -473,25 +474,31 @@ private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
override def sendSystemMessage(message: SystemMessage): Unit = { override def sendSystemMessage(message: SystemMessage): Unit = {
if (Mailbox.debug) println(s"ELAR $path having enqueued $message") 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 { override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = message match {
case null throw new InvalidMessageException("Message is null") case null throw new InvalidMessageException("Message is null")
case d: DeadLetter case d: DeadLetter
specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend! specialHandle(d.message, d.sender) // do NOT form endless loops, since deadLetters will resend!
case _ if !specialHandle(message) case _ if !specialHandle(message, sender)
eventStream.publish(DeadLetter(message, if (sender eq Actor.noSender) provider.deadLetters else sender, this)) eventStream.publish(DeadLetter(message, if (sender eq Actor.noSender) provider.deadLetters else sender, this))
case _ case _
} }
protected def specialHandle(msg: Any): Boolean = msg match { protected def specialHandle(msg: Any, sender: ActorRef): Boolean = msg match {
case w: Watch case w: Watch
if (w.watchee == this && w.watcher != this) if (w.watchee == this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false, addressTerminated = false) w.watcher ! Terminated(w.watchee)(existenceConfirmed = false, addressTerminated = false)
true true
case _: Unwatch true // Just ignore case _: Unwatch true // Just ignore
case _ false case Identify(messageId)
sender ! ActorIdentity(messageId, None)
true
case s: SelectChildName
s.identifyRequest foreach { x sender ! ActorIdentity(x.messageId, None) }
true
case _ false
} }
} }
@ -506,18 +513,25 @@ private[akka] class DeadLetterActorRef(_provider: ActorRefProvider,
_eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) { _eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) {
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match { override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
case null throw new InvalidMessageException("Message is null") case null throw new InvalidMessageException("Message is null")
case d: DeadLetter if (!specialHandle(d.message)) eventStream.publish(d) case Identify(messageId) sender ! ActorIdentity(messageId, Some(this))
case _ if (!specialHandle(message)) 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)) 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 case w: Watch
if (w.watchee != this && w.watcher != this) if (w.watchee != this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false, addressTerminated = false) w.watcher ! Terminated(w.watchee)(existenceConfirmed = false, addressTerminated = false)
true true
case w: Unwatch true // Just ignore 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 NullMessage true
case _ false case _ false
} }

View file

@ -4,6 +4,7 @@
package akka.actor package akka.actor
import scala.collection.immutable
import akka.dispatch.sysmsg._ import akka.dispatch.sysmsg._
import akka.dispatch.NullMessage import akka.dispatch.NullMessage
import akka.routing._ import akka.routing._
@ -28,6 +29,13 @@ trait ActorRefProvider {
*/ */
def rootGuardian: InternalActorRef 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. * 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 * Create actor reference for a specified local or remote path. If no such
* actor exists, it will be (equivalent to) a dead letter reference. * actor exists, it will be (equivalent to) a dead letter reference.
*/ */
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: ActorPath): InternalActorRef def actorFor(path: ActorPath): InternalActorRef
/** /**
@ -117,6 +126,7 @@ trait ActorRefProvider {
* (equivalent to) a dead letter reference. If `s` is a relative URI, resolve * (equivalent to) a dead letter reference. If `s` is a relative URI, resolve
* it relative to the given ref. * it relative to the given ref.
*/ */
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, s: String): InternalActorRef 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 * i.e. it cannot be used to obtain a reference to an actor which is not
* physically or logically attached to this actor system. * physically or logically attached to this actor system.
*/ */
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, p: Iterable[String]): InternalActorRef 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 * This Future is completed upon termination of this ActorRefProvider, which
* is usually initiated by stopping the guardian via ActorSystem.stop(). * 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 * `watch(ref)` to be notified of the targets termination, which is also
* signaled if the queried path cannot be resolved. * signaled if the queried path cannot be resolved.
*/ */
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: ActorPath): ActorRef = provider.actorFor(path) 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 * relative to the current context as described for look-ups by
* `actorOf(Iterable[String])` * `actorOf(Iterable[String])`
*/ */
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(path: String): ActorRef = provider.actorFor(lookupRoot, path) 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. * 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) 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. * 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)) 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 * the supplied path, it is recommended to send a message and gather the
* replies in order to resolve the matching set of actors. * 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 * 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 protected def systemGuardianStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy
lazy val rootGuardian: LocalActorRef = override lazy val rootGuardian: LocalActorRef =
new LocalActorRef(system, Props(new Guardian(rootGuardianStrategy)), theOneWhoWalksTheBubblesOfSpaceTime, rootPath) { new LocalActorRef(system, Props(new Guardian(rootGuardianStrategy)), theOneWhoWalksTheBubblesOfSpaceTime, rootPath) {
override def getParent: InternalActorRef = this override def getParent: InternalActorRef = this
override def getSingleChild(name: String): InternalActorRef = name match { override def getSingleChild(name: String): InternalActorRef = name match {
@ -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 val cell = rootGuardian.underlying
cell.reserveChild("user") cell.reserveChild("user")
val ref = new LocalActorRef(system, Props(new Guardian(guardianStrategy)), rootGuardian, rootPath / "user") val ref = new LocalActorRef(system, Props(new Guardian(guardianStrategy)), rootGuardian, rootPath / "user")
@ -524,7 +574,7 @@ private[akka] class LocalActorRefProvider private[akka] (
ref ref
} }
lazy val systemGuardian: LocalActorRef = { override lazy val systemGuardian: LocalActorRef = {
val cell = rootGuardian.underlying val cell = rootGuardian.underlying
cell.reserveChild("system") cell.reserveChild("system")
val ref = new LocalActorRef(system, Props(new SystemGuardian(systemGuardianStrategy)), rootGuardian, rootPath / "system") val ref = new LocalActorRef(system, Props(new SystemGuardian(systemGuardianStrategy)), rootGuardian, rootPath / "system")
@ -554,7 +604,8 @@ private[akka] class LocalActorRefProvider private[akka] (
eventStream.startDefaultLoggers(_system) 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) case RelativeActorPath(elems)
if (elems.isEmpty) { if (elems.isEmpty) {
log.debug("look-up of empty path string [{}] fails (per definition)", path) log.debug("look-up of empty path string [{}] fails (per definition)", path)
@ -567,14 +618,16 @@ private[akka] class LocalActorRefProvider private[akka] (
deadLetters 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) if (path.root == rootPath) actorFor(rootGuardian, path.elements)
else { else {
log.debug("look-up of foreign ActorPath [{}] failed", path) log.debug("look-up of foreign ActorPath [{}] failed", path)
deadLetters 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) { if (path.isEmpty) {
log.debug("look-up of empty path sequence fails (per definition)") log.debug("look-up of empty path sequence fails (per definition)")
deadLetters deadLetters
@ -585,6 +638,35 @@ private[akka] class LocalActorRefProvider private[akka] (
case x x 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, def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = { systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
props.routerConfig match { props.routerConfig match {

View file

@ -4,7 +4,7 @@
package akka.actor package akka.actor
import language.implicitConversions import language.implicitConversions
import scala.collection.immutable
import java.util.regex.Pattern import java.util.regex.Pattern
import akka.util.Helpers 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, * An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
* allowing for broadcasting of messages to that section. * allowing for broadcasting of messages to that section.
*/ */
abstract class ActorSelection { @SerialVersionUID(1L)
abstract class ActorSelection extends Serializable {
this: ScalaActorSelection this: ScalaActorSelection
protected def target: ActorRef protected def target: ActorRef
protected def path: Array[AnyRef] 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): Unit = target ! toMessage(msg, path)
def tell(msg: Any, sender: ActorRef): Unit = target.tell(toMessage(msg, path), sender) def tell(msg: Any, sender: ActorRef): Unit = target.tell(toMessage(msg, path), sender)
@ -36,6 +38,23 @@ abstract class ActorSelection {
} }
acc 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 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 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 def /(name: String): ActorPath
@ -269,7 +269,7 @@ abstract class ActorSystem extends ActorRefFactory {
def child(child: String): ActorPath = /(child) 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 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) indent + node.path.name + " " + Logging.simpleName(node)
} }
} }
printNode(actorFor("/"), "") printNode(lookupRoot, "")
} }
final class TerminationCallbacks extends Runnable with Awaitable[Unit] { 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)) 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. * 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(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)) 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) } 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? implicit val ec = camel.system.dispatcher // FIXME which ExecutionContext should be used here?
async.onComplete(action andThen { _ callback.done(false) }) 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 = private def fireAndForget(message: CamelMessage, exchange: CamelExchangeAdapter): Unit =
try { actorFor(endpoint.path) ! message } catch { case NonFatal(e) exchange.setFailure(new FailureResult(e)) } 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://")) require(actorPath.startsWith("akka://"))
def findActorIn(system: ActorSystem): Option[ActorRef] = { def findActorIn(system: ActorSystem): Option[ActorRef] = {
// FIXME #3074 how do we solve this with actorSelection?
val ref = system.actorFor(actorPath) val ref = system.actorFor(actorPath)
if (ref.isTerminated) None else Some(ref) 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 * This method is overridden here to keep track of remote deployed actors to
* be able to clean up corresponding child references. * be able to clean up corresponding child references.
*/ */
override def useActorOnNode(path: ActorPath, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = { override def useActorOnNode(ref: ActorRef, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
super.useActorOnNode(path, props, deploy, supervisor) super.useActorOnNode(ref, props, deploy, supervisor)
remoteDeploymentWatcher ! ((actorFor(path), supervisor)) import RemoteDeploymentWatcher.WatchRemote
remoteDeploymentWatcher ! WatchRemote(ref, supervisor)
} }
} }
/**
* INTERNAL API
*/
private[akka] object RemoteDeploymentWatcher {
case class WatchRemote(actor: ActorRef, supervisor: ActorRef)
}
/** /**
* INTERNAL API * INTERNAL API
* *
@ -84,10 +92,11 @@ private[akka] class ClusterActorRefProvider(
* goes down (jvm crash, network failure), i.e. triggered by [[akka.actor.AddressTerminated]]. * goes down (jvm crash, network failure), i.e. triggered by [[akka.actor.AddressTerminated]].
*/ */
private[akka] class RemoteDeploymentWatcher extends Actor { private[akka] class RemoteDeploymentWatcher extends Actor {
import RemoteDeploymentWatcher._
var supervisors = Map.empty[ActorRef, InternalActorRef] var supervisors = Map.empty[ActorRef, InternalActorRef]
def receive = { def receive = {
case (a: ActorRef, supervisor: InternalActorRef) case WatchRemote(a, supervisor: InternalActorRef)
supervisors += (a -> supervisor) supervisors += (a -> supervisor)
context.watch(a) context.watch(a)

View file

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

View file

@ -7,7 +7,7 @@ import language.postfixOps
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ 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.cluster.ClusterEvent._
import akka.routing.MurmurHash 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. * Looks up and returns the remote cluster heartbeat connection for the specific address.
*/ */
def heartbeatReceiver(address: Address): ActorRef = def heartbeatReceiver(address: Address): ActorSelection =
context.actorFor(RootActorPath(address) / "system" / "cluster" / "heartbeatReceiver") context.actorSelection(RootActorPath(address) / "system" / "cluster" / "heartbeatReceiver")
/** /**
* Looks up and returns the remote cluster heartbeat sender for the specific address. * 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 = { def receive = {
case HeartbeatTick heartbeat() case HeartbeatTick heartbeat()

View file

@ -157,7 +157,7 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto
sendGossip(address, MetricsGossipEnvelope(selfAddress, latestGossip, reply = true)) sendGossip(address, MetricsGossipEnvelope(selfAddress, latestGossip, reply = true))
def sendGossip(address: Address, envelope: MetricsGossipEnvelope): Unit = 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] = def selectRandomNode(addresses: immutable.IndexedSeq[Address]): Option[Address] =
if (addresses.isEmpty) None else Some(addresses(ThreadLocalRandom.current nextInt addresses.size)) 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 akka.remote.RemoteActorRef
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.actor.ActorSystemImpl import akka.actor.ActorSystemImpl
import akka.actor.ActorIdentity
import akka.actor.Identify
object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig { object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -65,13 +67,19 @@ abstract class ClusterDeathWatchSpec
val path2 = RootActorPath(second) / "user" / "subject" val path2 = RootActorPath(second) / "user" / "subject"
val path3 = RootActorPath(third) / "user" / "subject" val path3 = RootActorPath(third) / "user" / "subject"
val watchEstablished = TestLatch(1) val watchEstablished = TestLatch(2)
system.actorOf(Props(new Actor { system.actorOf(Props(new Actor {
context.watch(context.actorFor(path2)) context.actorSelection(path2) ! Identify(path2)
context.watch(context.actorFor(path3)) context.actorSelection(path3) ! Identify(path3)
watchEstablished.countDown
def receive = { 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") }), name = "observer1")

View file

@ -83,7 +83,7 @@ abstract class RestartFirstSeedNodeSpec
enterBarrier("seed1-address-receiver-ready") enterBarrier("seed1-address-receiver-ready")
seedNode1Address = Cluster(seed1System).selfAddress seedNode1Address = Cluster(seed1System).selfAddress
List(seed2, seed3) foreach { r 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") expectMsg(5 seconds, "ok")
} }
} }

View file

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

View file

@ -14,32 +14,32 @@ Some examples:
* single master, many workers * single master, many workers
* centralized naming service, or routing logic * centralized naming service, or routing logic
Using a singleton should not be the first design choice. It has several drawbacks, Using a singleton should not be the first design choice. It has several drawbacks,
such as single-point of bottleneck. Single-point of failure is also a relevant concern, such as single-point of bottleneck. Single-point of failure is also a relevant concern,
but for some cases this feature takes care of that by making sure that another singleton but for some cases this feature takes care of that by making sure that another singleton
instance will eventually be started. instance will eventually be started.
The cluster singleton pattern is implemented by ``akka.contrib.pattern.ClusterSingletonManager``. The cluster singleton pattern is implemented by ``akka.contrib.pattern.ClusterSingletonManager``.
It manages singleton actor instance among all cluster nodes or a group of nodes tagged with It manages singleton actor instance among all cluster nodes or a group of nodes tagged with
a specific role. ``ClusterSingletonManager`` is an actor that is supposed to be started on a specific role. ``ClusterSingletonManager`` is an actor that is supposed to be started on
all nodes, or all nodes with specified role, in the cluster. The actual singleton actor is all nodes, or all nodes with specified role, in the cluster. The actual singleton actor is
started by the ``ClusterSingletonManager`` on the leader node by creating a child actor from started by the ``ClusterSingletonManager`` on the leader node by creating a child actor from
supplied ``Props``. ``ClusterSingletonManager`` makes sure that at most one singleton instance supplied ``Props``. ``ClusterSingletonManager`` makes sure that at most one singleton instance
is running at any point in time. is running at any point in time.
The singleton actor is always running on the leader member, which is nothing more than The singleton actor is always running on the leader member, which is nothing more than
the address currently sorted first in the member ring. This can change when adding the address currently sorted first in the member ring. This can change when adding
or removing members. A graceful hand over can normally be performed when joining a new or removing members. A graceful hand over can normally be performed when joining a new
node that becomes leader or removing current leader node. Be aware that there is a short node that becomes leader or removing current leader node. Be aware that there is a short
time period when there is no active singleton during the hand over process. time period when there is no active singleton during the hand over process.
The cluster failure detector will notice when a leader node becomes unreachable due to The cluster failure detector will notice when a leader node becomes unreachable due to
things like JVM crash, hard shut down, or network failure. Then a new leader node will things like JVM crash, hard shut down, or network failure. Then a new leader node will
take over and a new singleton actor is created. For these failure scenarios there will take over and a new singleton actor is created. For these failure scenarios there will
not be a graceful hand-over, but more than one active singletons is prevented by all 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. 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 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. ``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. Alternatively the singleton actor may broadcast its existence when it is started.
@ -47,13 +47,13 @@ Alternatively the singleton actor may broadcast its existence when it is started
An Example An Example
---------- ----------
Assume that we need one single entry point to an external system. An actor that Assume that we need one single entry point to an external system. An actor that
receives messages from a JMS queue with the strict requirement that only one receives messages from a JMS queue with the strict requirement that only one
JMS consumer must exist to be make sure that the messages are processed in order. JMS consumer must exist to be make sure that the messages are processed in order.
That is perhaps not how one would like to design things, but a typical real-world That is perhaps not how one would like to design things, but a typical real-world
scenario when integrating with external systems. scenario when integrating with external systems.
On each node in the cluster you need to start the ``ClusterSingletonManager`` and On each node in the cluster you need to start the ``ClusterSingletonManager`` and
supply the ``Props`` of the singleton actor, in this case the JMS queue consumer. supply the ``Props`` of the singleton actor, in this case the JMS queue consumer.
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#create-singleton-manager .. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#create-singleton-manager
@ -66,7 +66,7 @@ The Java API constructor takes a plain String for the role parameter and ``null`
role, are used. role, are used.
Here we use an application specific ``terminationMessage`` to be able to close the Here we use an application specific ``terminationMessage`` to be able to close the
resources before actually stopping the singleton actor. Note that ``PoisonPill`` is a resources before actually stopping the singleton actor. Note that ``PoisonPill`` is a
perfectly fine ``terminationMessage`` if you only need to stop the actor. perfectly fine ``terminationMessage`` if you only need to stop the actor.
Here is how the singleton actor handles the ``terminationMessage`` in this example. Here is how the singleton actor handles the ``terminationMessage`` in this example.
@ -77,25 +77,25 @@ Note that you can send back current state to the ``ClusterSingletonManager`` bef
This message will be sent over to the ``ClusterSingletonManager`` at the new leader node and it This message will be sent over to the ``ClusterSingletonManager`` at the new leader node and it
will be passed to the ``singletonProps`` factory when creating the new singleton instance. 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 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 .. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#singleton-proxy2
Subscribe to ``LeaderChanged`` instead of ``RoleLeaderChanged`` if you don't limit the singleton to Subscribe to ``LeaderChanged`` instead of ``RoleLeaderChanged`` if you don't limit the singleton to
the group of members tagged with a specific role. the group of members tagged with a specific role.
Note that the hand-over might still be in progress and the singleton actor might not be started yet Note that the hand-over might still be in progress and the singleton actor might not be started yet
when you receive the ``LeaderChanged`` / ``RoleLeaderChanged`` event. when you receive the ``LeaderChanged`` / ``RoleLeaderChanged`` event.
To test scenarios where the cluster leader node is removed or shut down you can use :ref:`multi-node-testing` and To test scenarios where the cluster leader node is removed or shut down you can use :ref:`multi-node-testing` and
utilize the fact that the leader is supposed to be the first member when sorted by member address. utilize the fact that the leader is supposed to be the first member when sorted by member address.
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#sort-cluster-roles .. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#sort-cluster-roles
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#test-leave .. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterSingletonManagerSpec.scala#test-leave
Also, make sure that you don't shut down the first role, which is running the test conductor controller. Also, make sure that you don't shut down the first role, which is running the test conductor controller.
Use a dedicated role for the controller, which is not a cluster member. Use a dedicated role for the controller, which is not a cluster member.
.. note:: The singleton pattern will be simplified, perhaps provided out-of-the-box, when the cluster handles automatic actor partitioning. .. note:: The singleton pattern will be simplified, perhaps provided out-of-the-box, when the cluster handles automatic actor partitioning.

View file

@ -9,6 +9,7 @@ import akka.actor.Actor
import akka.actor.Actor.Receive import akka.actor.Actor.Receive
import akka.actor.ActorLogging import akka.actor.ActorLogging
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.ActorSelection
import akka.actor.Address import akka.actor.Address
import akka.actor.FSM import akka.actor.FSM
import akka.actor.Props 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 * is prevented by all reasonable means. Some corner cases are eventually
* resolved by configurable timeouts. * 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 * specified when creating the ClusterSingletonManager. You can subscribe to
* [[akka.cluster.ClusterEvent.LeaderChanged]] or * [[akka.cluster.ClusterEvent.LeaderChanged]] or
* [[akka.cluster.ClusterEvent.RoleLeaderChanged]] to keep track of which node * [[akka.cluster.ClusterEvent.RoleLeaderChanged]] to keep track of which node
@ -385,7 +386,7 @@ class ClusterSingletonManager(
super.postStop() 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 = def getNextLeaderChanged(): Unit =
if (leaderChangedReceived) { if (leaderChangedReceived) {

View file

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

View file

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

View file

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

View file

@ -95,14 +95,14 @@ What is the Difference Between Actor Reference and Path?
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
An actor reference designates a single actor and the life-cycle of the reference An actor reference designates a single actor and the life-cycle of the reference
matches that actors life-cycle; an actor path represents a name which may or matches that actors life-cycle; an actor path represents a name which may or
may not be inhabited by an actor and the path itself does not have a life-cycle, may not be inhabited by an actor and the path itself does not have a life-cycle,
it never becomes invalid. You can create an actor path without creating an actor, it never becomes invalid. You can create an actor path without creating an actor,
but you cannot create an actor reference without creating corresponding actor. but you cannot create an actor reference without creating corresponding actor.
.. note:: .. note::
That definition does not hold for ``actorFor``, which is one of the reasons why That definition does not hold for ``actorFor``, which is one of the reasons why
``actorFor`` is deprecated in favor of ``actorSelection``. ``actorFor`` is deprecated in favor of ``actorSelection``.
You can create an actor, terminate it, and then create a new actor with the same You can create an actor, terminate it, and then create a new actor with the same
@ -194,34 +194,43 @@ Looking up Actors by Concrete Path
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
In addition, actor references may be looked up using the In addition, actor references may be looked up using the
:meth:`ActorSystem.actorFor` method, which returns a local or remote actor :meth:`ActorSystem.actorSelection` method. The selection can be used for
reference. The reference can be reused for communicating with said actor during communicating with said actor and the actor corresponding to the selection
the whole lifetime of the actor. In the case of a local actor reference, the is looked up when delivering each message.
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 To acquire an :class:`ActorRef` that is bound to the life-cycle of a specific actor
that exact path is created after acquiring the actor reference. For remote actor you need to send a message, such as the built-in :class:`Identify` message, to the actor
references acquired with `actorFor` the behaviour is different and sending messages and use the ``sender`` reference of a reply from the actor.
to such a reference will under the hood look up the actor by path on the remote
system for every message send. .. 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 Absolute vs. Relative Paths
``````````````````````````` ```````````````````````````
In addition to :meth:`ActorSystem.actorFor` there is also In addition to :meth:`ActorSystem.actorSelection` there is also
:meth:`ActorContext.actorFor`, which is available inside any actor as :meth:`ActorContext.actorSelection`, which is available inside any actor as
``context.actorFor``. This yields an actor reference much like its twin on ``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 :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 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 of two dots (``".."``) may be used to access the parent actor. You can for
example send a message to a specific sibling:: 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. Absolute paths may of course also be looked up on `context` in the usual way, i.e.
.. code-block:: scala .. code-block:: scala
context.actorFor("/user/serviceA") ! msg context.actorSelection("/user/serviceA") ! msg
will work as expected. 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 actors. This scheme of resolving a selection may be improved upon in a future
release. release.
.. _actorOf-vs-actorFor: .. _actorOf-vs-actorSelection:
Summary: ``actorOf`` vs. ``actorFor`` Summary: ``actorOf`` vs. ``actorSelection`` vs. ``actorFor``
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
.. note:: .. note::
@ -263,8 +272,12 @@ Summary: ``actorOf`` vs. ``actorFor``
child of the context on which this method is invoked (which may be any child of the context on which this method is invoked (which may be any
actor or actor system). actor or actor system).
- ``actorFor`` only ever looks up an existing actor, i.e. does not create - ``actorSelection`` only ever looks up existing actors when messages are
one. 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 Actor Reference and Path Equality
--------------------------------- ---------------------------------
@ -273,13 +286,13 @@ Equality of ``ActorRef`` match the intention that an ``ActorRef`` corresponds to
the target actor incarnation. Two actor references are compared equal when they have the target actor incarnation. Two actor references are compared equal when they have
the same path and point to the same actor incarnation. A reference pointing to a the same path and point to the same actor incarnation. A reference pointing to a
terminated actor does not compare equal to a reference pointing to another (re-created) terminated actor does not compare equal to a reference pointing to another (re-created)
actor with the same path. Note that a restart of an actor caused by a failure still actor with the same path. Note that a restart of an actor caused by a failure still
means that it is the same actor incarnation, i.e. a restart is not visible for the means that it is the same actor incarnation, i.e. a restart is not visible for the
consumer of the ``ActorRef``. consumer of the ``ActorRef``.
Remote actor references acquired with ``actorFor`` do not include the full Remote actor references acquired with ``actorFor`` do not include the full
information about the underlying actor identity and therefore such references information about the underlying actor identity and therefore such references
do not compare equal to references acquired with ``actorOf``, ``sender``, do not compare equal to references acquired with ``actorOf``, ``sender``,
or ``context.self``. Because of this ``actorFor`` is deprecated in favor of or ``context.self``. Because of this ``actorFor`` is deprecated in favor of
``actorSelection``. ``actorSelection``.
@ -297,7 +310,7 @@ While it is possible to create an actor at a later time with an identical
path—simply due to it being impossible to enforce the opposite without keeping path—simply due to it being impossible to enforce the opposite without keeping
the set of all actors ever created available—this is not good practice: remote the set of all actors ever created available—this is not good practice: remote
actor references acquired with ``actorFor`` which “died” suddenly start to work actor references acquired with ``actorFor`` which “died” suddenly start to work
again, but without any guarantee of ordering between this transition and any again, but without any guarantee of ordering between this transition and any
other event, hence the new inhabitant of the path may receive messages which were destined for the other event, hence the new inhabitant of the path may receive messages which were destined for the
previous tenant. previous tenant.

View file

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

View file

@ -15,6 +15,7 @@ import akka.dispatch.Futures;
import akka.dispatch.Mapper; import akka.dispatch.Mapper;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
import akka.testkit.AkkaSpec;
import akka.util.Timeout; import akka.util.Timeout;
//#import-future //#import-future
@ -31,6 +32,12 @@ import akka.japi.Procedure;
import akka.actor.Terminated; import akka.actor.Terminated;
//#import-watch //#import-watch
//#import-identify
import akka.actor.ActorSelection;
import akka.actor.Identify;
import akka.actor.ActorIdentity;
//#import-identify
//#import-gracefulStop //#import-gracefulStop
import static akka.pattern.Patterns.gracefulStop; import static akka.pattern.Patterns.gracefulStop;
import scala.concurrent.Future; import scala.concurrent.Future;
@ -58,6 +65,8 @@ import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import org.junit.Test; import org.junit.Test;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import scala.Option; import scala.Option;
import java.lang.Object; import java.lang.Object;
import java.util.Iterator; import java.util.Iterator;
@ -65,6 +74,19 @@ import akka.pattern.Patterns;
public class UntypedActorDocTestBase { 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 @Test
public void createProps() { public void createProps() {
//#creating-props-config //#creating-props-config
@ -96,86 +118,71 @@ public class UntypedActorDocTestBase {
@Test @Test
public void contextActorOf() { public void contextActorOf() {
//#context-actorOf //#context-actorOf
ActorSystem system = ActorSystem.create("MySystem"); ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class), "myactor2");
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class), "myactor");
//#context-actorOf //#context-actorOf
myActor.tell("test", null); myActor.tell("test", null);
system.shutdown();
} }
@Test @Test
public void constructorActorOf() { public void constructorActorOf() {
ActorSystem system = ActorSystem.create("MySystem");
//#creating-constructor //#creating-constructor
// allows passing in arguments to the MyActor constructor // allows passing in arguments to the MyActor constructor
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() { ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
public UntypedActor create() { public UntypedActor create() {
return new MyActor("..."); return new MyActor("...");
} }
}), "myactor"); }), "myactor3");
//#creating-constructor //#creating-constructor
myActor.tell("test", null); myActor.tell("test", null);
system.shutdown();
} }
@Test @Test
public void propsActorOf() { public void propsActorOf() {
ActorSystem system = ActorSystem.create("MySystem");
//#creating-props //#creating-props
ActorRef myActor = system.actorOf( ActorRef myActor = system.actorOf(
new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), "myactor"); new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), "myactor4");
//#creating-props //#creating-props
myActor.tell("test", null); myActor.tell("test", null);
system.shutdown();
} }
@Test @Test
public void usingAsk() throws Exception { public void usingAsk() throws Exception {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() { ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
public UntypedActor create() { public UntypedActor create() {
return new MyAskActor(); return new MyAskActor();
} }
}), "myactor"); }), "myactor5");
//#using-ask //#using-ask
Future<Object> future = Patterns.ask(myActor, "Hello", 1000); Future<Object> future = Patterns.ask(myActor, "Hello", 1000);
Object result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)); Object result = Await.result(future, Duration.create(1, TimeUnit.SECONDS));
//#using-ask //#using-ask
system.shutdown();
} }
@Test @Test
public void receiveTimeout() { public void receiveTimeout() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(MyReceivedTimeoutUntypedActor.class)); ActorRef myActor = system.actorOf(new Props(MyReceivedTimeoutUntypedActor.class));
myActor.tell("Hello", null); myActor.tell("Hello", null);
system.shutdown();
} }
@Test @Test
public void usePoisonPill() { public void usePoisonPill() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class)); ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class));
//#poison-pill //#poison-pill
myActor.tell(PoisonPill.getInstance(), null); myActor.tell(PoisonPill.getInstance(), null);
//#poison-pill //#poison-pill
system.shutdown();
} }
@Test @Test
public void useKill() { public void useKill() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef victim = system.actorOf(new Props(MyUntypedActor.class)); ActorRef victim = system.actorOf(new Props(MyUntypedActor.class));
//#kill //#kill
victim.tell(Kill.getInstance(), null); victim.tell(Kill.getInstance(), null);
//#kill //#kill
system.shutdown();
} }
@Test @Test
public void useBecome() { public void useBecome() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() { ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
public UntypedActor create() { public UntypedActor create() {
return new HotSwapActor(); return new HotSwapActor();
@ -184,21 +191,24 @@ public class UntypedActorDocTestBase {
myActor.tell("foo", null); myActor.tell("foo", null);
myActor.tell("bar", null); myActor.tell("bar", null);
myActor.tell("bar", null); myActor.tell("bar", null);
system.shutdown();
} }
@Test @Test
public void useWatch() throws Exception { public void useWatch() throws Exception {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(new Props(WatchActor.class)); ActorRef myActor = system.actorOf(new Props(WatchActor.class));
Future<Object> future = Patterns.ask(myActor, "kill", 1000); Future<Object> future = Patterns.ask(myActor, "kill", 1000);
assert Await.result(future, Duration.create("1 second")).equals("finished"); assert Await.result(future, Duration.create("1 second")).equals("finished");
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 @Test
public void usePatternsGracefulStop() throws Exception { public void usePatternsGracefulStop() throws Exception {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef actorRef = system.actorOf(new Props(MyUntypedActor.class)); ActorRef actorRef = system.actorOf(new Props(MyUntypedActor.class));
//#gracefulStop //#gracefulStop
try { try {
@ -210,7 +220,6 @@ public class UntypedActorDocTestBase {
// the actor wasn't stopped within 5 seconds // the actor wasn't stopped within 5 seconds
} }
//#gracefulStop //#gracefulStop
system.shutdown();
} }
class Result { class Result {
@ -225,7 +234,6 @@ public class UntypedActorDocTestBase {
@Test @Test
public void usePatternsAskPipe() { public void usePatternsAskPipe() {
ActorSystem system = ActorSystem.create("MySystem");
ActorRef actorA = system.actorOf(new Props(MyUntypedActor.class)); ActorRef actorA = system.actorOf(new Props(MyUntypedActor.class));
ActorRef actorB = system.actorOf(new Props(MyUntypedActor.class)); ActorRef actorB = system.actorOf(new Props(MyUntypedActor.class));
ActorRef actorC = 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); pipe(transformed, system.dispatcher()).to(actorC);
//#ask-pipe //#ask-pipe
system.shutdown();
} }
public static class MyActor extends UntypedActor { public static class MyActor extends UntypedActor {
@ -399,4 +406,40 @@ public class UntypedActorDocTestBase {
} }
//#watch //#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 //#my-own-serializer
@Test public void serializeActorRefs() { @Test public void serializeActorRefs() {
final ActorSystem theActorSystem = final ExtendedActorSystem extendedSystem = (ExtendedActorSystem)
ActorSystem.create("whatever"); ActorSystem.create("whatever");
final ActorRef theActorRef = final ActorRef theActorRef =
theActorSystem.deadLetters(); // Of course this should be you extendedSystem.deadLetters(); // Of course this should be you
//#actorref-serializer //#actorref-serializer
// Serialize // Serialize
@ -63,10 +63,10 @@ public class SerializationDocTestBase {
// Deserialize // Deserialize
// (beneath fromBinary) // (beneath fromBinary)
final ActorRef deserializedActorRef = theActorSystem.actorFor(identifier); final ActorRef deserializedActorRef = extendedSystem.provider().resolveActorRef(identifier);
// Then just use the ActorRef // Then just use the ActorRef
//#actorref-serializer //#actorref-serializer
theActorSystem.shutdown(); extendedSystem.shutdown();
} }
static static
@ -150,7 +150,7 @@ public class SerializationDocTestBase {
return new DefaultAddressExt(system); return new DefaultAddressExt(system);
} }
} }
//#external-address-default //#external-address-default
public void demonstrateDefaultAddress() { public void demonstrateDefaultAddress() {

View file

@ -16,6 +16,7 @@ import akka.actor.ActorKilledException;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.Kill; import akka.actor.Kill;
import akka.actor.PoisonPill;
import akka.actor.Props; import akka.actor.Props;
import akka.actor.Terminated; import akka.actor.Terminated;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
@ -30,7 +31,7 @@ import akka.testkit.JavaTestKit;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
public class TestKitDocTest { public class TestKitDocTest {
//#test-actor-ref //#test-actor-ref
static class MyActor extends UntypedActor { static class MyActor extends UntypedActor {
public void onReceive(Object o) throws Exception { public void onReceive(Object o) throws Exception {
@ -42,18 +43,18 @@ public class TestKitDocTest {
} }
public boolean testMe() { return true; } public boolean testMe() { return true; }
} }
//#test-actor-ref //#test-actor-ref
private static ActorSystem system; private static ActorSystem system;
@BeforeClass @BeforeClass
public static void setup() { public static void setup() {
final Config config = ConfigFactory.parseString( final Config config = ConfigFactory.parseString(
"akka.loggers = [akka.testkit.TestEventListener]"); "akka.loggers = [akka.testkit.TestEventListener]");
system = ActorSystem.create("demoSystem", config); system = ActorSystem.create("demoSystem", config);
} }
@AfterClass @AfterClass
public static void cleanup() { public static void cleanup() {
system.shutdown(); system.shutdown();
@ -68,7 +69,7 @@ public class TestKitDocTest {
assertTrue(actor.testMe()); assertTrue(actor.testMe());
} }
//#test-actor-ref //#test-actor-ref
@Test @Test
public void demonstrateAsk() throws Exception { public void demonstrateAsk() throws Exception {
//#test-behavior //#test-behavior
@ -79,7 +80,7 @@ public class TestKitDocTest {
assertEquals(42, Await.result(future, Duration.Zero())); assertEquals(42, Await.result(future, Duration.Zero()));
//#test-behavior //#test-behavior
} }
@Test @Test
public void demonstrateExceptions() { public void demonstrateExceptions() {
//#test-expecting-exceptions //#test-expecting-exceptions
@ -93,7 +94,7 @@ public class TestKitDocTest {
} }
//#test-expecting-exceptions //#test-expecting-exceptions
} }
@Test @Test
public void demonstrateWithin() { public void demonstrateWithin() {
//#test-within //#test-within
@ -108,7 +109,7 @@ public class TestKitDocTest {
}}; }};
//#test-within //#test-within
} }
@Test @Test
public void demonstrateExpectMsg() { public void demonstrateExpectMsg() {
//#test-expectmsg //#test-expectmsg
@ -128,7 +129,7 @@ public class TestKitDocTest {
}}; }};
//#test-expectmsg //#test-expectmsg
} }
@Test @Test
public void demonstrateReceiveWhile() { public void demonstrateReceiveWhile() {
//#test-receivewhile //#test-receivewhile
@ -136,7 +137,7 @@ public class TestKitDocTest {
getRef().tell(42, null); getRef().tell(42, null);
getRef().tell(43, null); getRef().tell(43, null);
getRef().tell("hello", null); getRef().tell("hello", null);
final String[] out = final String[] out =
new ReceiveWhile<String>(String.class, duration("1 second")) { new ReceiveWhile<String>(String.class, duration("1 second")) {
// do not put code outside this method, will run afterwards // do not put code outside this method, will run afterwards
protected String match(Object in) { protected String match(Object in) {
@ -168,7 +169,7 @@ public class TestKitDocTest {
//#test-receivewhile-full //#test-receivewhile-full
}}; }};
} }
@Test @Test
public void demonstrateAwaitCond() { public void demonstrateAwaitCond() {
//#test-awaitCond //#test-awaitCond
@ -205,7 +206,7 @@ public class TestKitDocTest {
}}; }};
//#test-awaitAssert //#test-awaitAssert
} }
@Test @Test
@SuppressWarnings("unchecked") // due to generic varargs @SuppressWarnings("unchecked") // due to generic varargs
public void demonstrateExpect() { public void demonstrateExpect() {
@ -236,7 +237,7 @@ public class TestKitDocTest {
assertArrayEquals(new String[] {"hello", "world"}, all); assertArrayEquals(new String[] {"hello", "world"}, all);
}}; }};
} }
@Test @Test
public void demonstrateIgnoreMsg() { public void demonstrateIgnoreMsg() {
//#test-ignoreMsg //#test-ignoreMsg
@ -268,7 +269,7 @@ public class TestKitDocTest {
}}; }};
//#duration-dilation //#duration-dilation
} }
@Test @Test
public void demonstrateProbe() { public void demonstrateProbe() {
//#test-probe //#test-probe
@ -282,11 +283,11 @@ public class TestKitDocTest {
target.forward(msg, getContext()); target.forward(msg, getContext());
} }
} }
new JavaTestKit(system) {{ new JavaTestKit(system) {{
// create a test probe // create a test probe
final JavaTestKit probe = new JavaTestKit(system); final JavaTestKit probe = new JavaTestKit(system);
// create a forwarder, injecting the probes testActor // create a forwarder, injecting the probes testActor
final Props props = new Props(new UntypedActorFactory() { final Props props = new Props(new UntypedActorFactory() {
private static final long serialVersionUID = 8927158735963950216L; private static final long serialVersionUID = 8927158735963950216L;
@ -295,7 +296,7 @@ public class TestKitDocTest {
} }
}); });
final ActorRef forwarder = system.actorOf(props, "forwarder"); final ActorRef forwarder = system.actorOf(props, "forwarder");
// verify correct forwarding // verify correct forwarding
forwarder.tell(42, getRef()); forwarder.tell(42, getRef());
probe.expectMsgEquals(42); probe.expectMsgEquals(42);
@ -303,7 +304,7 @@ public class TestKitDocTest {
}}; }};
//#test-probe //#test-probe
} }
@Test @Test
public void demonstrateSpecialProbe() { public void demonstrateSpecialProbe() {
//#test-special-probe //#test-special-probe
@ -323,20 +324,21 @@ public class TestKitDocTest {
}}; }};
//#test-special-probe //#test-special-probe
} }
@Test @Test
public void demonstrateWatch() { public void demonstrateWatch() {
final ActorRef target = system.actorFor("/buh"); final ActorRef target = system.actorOf(new Props(MyActor.class));
//#test-probe-watch //#test-probe-watch
new JavaTestKit(system) {{ new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system); final JavaTestKit probe = new JavaTestKit(system);
probe.watch(target); probe.watch(target);
target.tell(PoisonPill.getInstance(), null);
final Terminated msg = probe.expectMsgClass(Terminated.class); final Terminated msg = probe.expectMsgClass(Terminated.class);
assertEquals(msg.getActor(), target); assertEquals(msg.getActor(), target);
}}; }};
//#test-probe-watch //#test-probe-watch
} }
@Test @Test
public void demonstrateReply() { public void demonstrateReply() {
//#test-probe-reply //#test-probe-reply
@ -350,7 +352,7 @@ public class TestKitDocTest {
}}; }};
//#test-probe-reply //#test-probe-reply
} }
@Test @Test
public void demonstrateForward() { public void demonstrateForward() {
//#test-probe-forward //#test-probe-forward
@ -364,7 +366,7 @@ public class TestKitDocTest {
}}; }};
//#test-probe-forward //#test-probe-forward
} }
@Test @Test
public void demonstrateWithinProbe() { public void demonstrateWithinProbe() {
try { try {
@ -382,7 +384,7 @@ public class TestKitDocTest {
// expected to fail // expected to fail
} }
} }
@Test @Test
public void demonstrateAutoPilot() { public void demonstrateAutoPilot() {
//#test-auto-pilot //#test-auto-pilot
@ -404,7 +406,7 @@ public class TestKitDocTest {
}}; }};
//#test-auto-pilot //#test-auto-pilot
} }
// only compilation // only compilation
public void demonstrateCTD() { public void demonstrateCTD() {
//#calling-thread-dispatcher //#calling-thread-dispatcher
@ -413,24 +415,24 @@ public class TestKitDocTest {
.withDispatcher(CallingThreadDispatcher.Id())); .withDispatcher(CallingThreadDispatcher.Id()));
//#calling-thread-dispatcher //#calling-thread-dispatcher
} }
@Test @Test
public void demonstrateEventFilter() { public void demonstrateEventFilter() {
//#test-event-filter //#test-event-filter
new JavaTestKit(system) {{ new JavaTestKit(system) {{
assertEquals("demoSystem", system.name()); assertEquals("demoSystem", system.name());
final ActorRef victim = system.actorOf(Props.empty(), "victim"); final ActorRef victim = system.actorOf(Props.empty(), "victim");
final int result = new EventFilter<Integer>(ActorKilledException.class) { final int result = new EventFilter<Integer>(ActorKilledException.class) {
protected Integer run() { protected Integer run() {
victim.tell(Kill.getInstance(), null); victim.tell(Kill.getInstance(), null);
return 42; return 42;
} }
}.from("akka://demoSystem/user/victim").occurrences(1).exec(); }.from("akka://demoSystem/user/victim").occurrences(1).exec();
assertEquals(42, result); assertEquals(42, result);
}}; }};
//#test-event-filter //#test-event-filter
} }
} }

View file

@ -70,17 +70,24 @@ reference file for more information:
Looking up Remote Actors 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> 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:: .. 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 .. includecode:: ../../../akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApplication.java
:include: setup :include: setup
Requests which come in via ``doSomething`` will be sent to the client actor 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 which will use the actor reference that was identified earlier. Observe how the actor
system name using in ``actorFor`` matches the remote systems name, as do IP 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"`` and port number. Top-level actors are always created below the ``"/user"``
guardian, which supervises them. guardian, which supervises them.
@ -481,14 +488,14 @@ SSL can be used as the remote transport by adding ``akka.remote.netty.ssl``
to the ``enabled-transport`` configuration section. See a description of the settings to the ``enabled-transport`` configuration section. See a description of the settings
in the :ref:`remoting-java-configuration` section. in the :ref:`remoting-java-configuration` section.
The SSL support is implemented with Java Secure Socket Extension, please consult the offical The SSL support is implemented with Java Secure Socket Extension, please consult the offical
`Java Secure Socket Extension documentation <http://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html>`_ `Java Secure Socket Extension documentation <http://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html>`_
and related resources for troubleshooting. and related resources for troubleshooting.
.. note:: .. note::
When using SHA1PRNG on Linux it's recommended specify ``-Djava.security.egd=file:/dev/./urandom`` as argument When using SHA1PRNG on Linux it's recommended specify ``-Djava.security.egd=file:/dev/./urandom`` as argument
to the JVM to prevent blocking. It is NOT as secure because it reuses the seed. to the JVM to prevent blocking. It is NOT as secure because it reuses the seed.
Use '/dev/./urandom', not '/dev/urandom' as that doesn't work according to Use '/dev/./urandom', not '/dev/urandom' as that doesn't work according to
`Bug ID: 6202721 <http://bugs.sun.com/view_bug.do?bug_id=6202721>`_. `Bug ID: 6202721 <http://bugs.sun.com/view_bug.do?bug_id=6202721>`_.

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. 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 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 .. 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 Since Akka enforces parental supervision every actor is supervised and
(potentially) the supervisor of its children, it is advisable that you (potentially) the supervisor of its children, it is advisable that you
familiarize yourself with :ref:`actor-systems` and :ref:`supervision` and it familiarize yourself with :ref:`actor-systems` and :ref:`supervision` and it
may also help to read :ref:`actorOf-vs-actorFor` (the whole of may also help to read :ref:`addressing`.
:ref:`addressing` is recommended reading in any case).
Defining an Actor class Defining an Actor class
----------------------- -----------------------
@ -130,7 +129,7 @@ The :class:`UntypedActor` class defines only one abstract method, the above ment
If the current actor behavior does not match a received message, it's recommended that If the current actor behavior does not match a received message, it's recommended that
you call the :meth:`unhandled` method, which by default publishes a ``new you call the :meth:`unhandled` method, which by default publishes a ``new
akka.actor.UnhandledMessage(message, sender, recipient)`` on the actor systems akka.actor.UnhandledMessage(message, sender, recipient)`` on the actor systems
event stream (set configuration item ``akka.actor.debug.unhandled`` to ``on`` event stream (set configuration item ``akka.actor.debug.unhandled`` to ``on``
to have them converted into actual Debug messages). to have them converted into actual Debug messages).
In addition, it offers: In addition, it offers:
@ -227,7 +226,7 @@ mentioned above:
in turn by its supervisor, or if an actor is restarted due to a siblings in turn by its supervisor, or if an actor is restarted due to a siblings
failure. If the message is available, then that messages sender is also failure. If the message is available, then that messages sender is also
accessible in the usual way (i.e. by calling ``getSender()``). accessible in the usual way (i.e. by calling ``getSender()``).
This method is the best place for cleaning up, preparing hand-over to the This method is the best place for cleaning up, preparing hand-over to the
fresh actor instance, etc. By default it stops all children and calls fresh actor instance, etc. By default it stops all children and calls
:meth:`postStop`. :meth:`postStop`.
@ -264,8 +263,10 @@ sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
:obj:`ActorSystem`. :obj:`ActorSystem`.
Identifying Actors .. _actorSelection-java:
==================
Identifying Actors via Actor Selection
======================================
As described in :ref:`addressing`, each actor has a unique logical path, which 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 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 are used by the system to look up actors, e.g. when a remote message is
received and the recipient is searched, but they are also useful more directly: received and the recipient is searched, but they are also useful more directly:
actors may look up other actors by specifying absolute or relative actors may look up other actors by specifying absolute or relative
paths—logical or physical—and receive back an :class:`ActorRef` with the paths—logical or physical—and receive back an :class:`ActorSelection` with the
result:: result::
getContext().actorFor("/user/serviceA/actor") // will look up this absolute path // will look up this absolute path
getContext().actorFor("../joe") // will look up sibling beneath same supervisor getContext().actorSelection("/user/serviceA/actor");
// will look up sibling beneath same supervisor
getContext().actorSelection("../joe");
The supplied path is parsed as a :class:`java.net.URI`, which basically means The supplied path is parsed as a :class:`java.net.URI`, which basically means
that it is split on ``/`` into path elements. If the path starts with ``/``, it that it is split on ``/`` into path elements. If the path starts with ``/``, it
@ -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 It should be noted that the ``..`` in actor paths here always means the logical
structure, i.e. the supervisor. structure, i.e. the supervisor.
If the path being looked up does not exist, a special actor reference is The path elements of an actor selection may contain wildcard patterns allowing for
returned which behaves like the actor systems dead letter queue but retains broadcasting of messages to that section::
its identity (i.e. the path which was looked up).
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 To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
will have to send to it and await a reply in order to verify that ``serviceB`` send a message to the selection and use the ``getSender`` reference of the reply from
is actually reachable and running. An example demonstrating actor look-up is the actor. There is a built-in ``Identify`` message that all Actors will understand
given in :ref:`remote-lookup-sample-java`. 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 Messages and immutability
========================= =========================
@ -603,7 +631,7 @@ The other way of using :meth:`become` does not replace but add to the top of
the behavior stack. In this case care must be taken to ensure that the number the behavior stack. In this case care must be taken to ensure that the number
of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones
in the long run, otherwise this amounts to a memory leak (which is why this in the long run, otherwise this amounts to a memory leak (which is why this
behavior is not the default). behavior is not the default).
.. includecode:: code/docs/actor/UntypedActorSwapper.java#swapper .. includecode:: code/docs/actor/UntypedActorSwapper.java#swapper

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` 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 ActorRef equality and sending to remote actors
============================================== ==============================================
@ -159,15 +186,15 @@ Sending messages to an ``ActorRef`` must have the same semantics no matter if th
on a remote host or in the same ``ActorSystem`` in the same JVM. This was not always the case. For example on a remote host or in the same ``ActorSystem`` in the same JVM. This was not always the case. For example
when the target actor is terminated and created again under the same path. Sending to local references when the target actor is terminated and created again under the same path. Sending to local references
of the previous incarnation of the actor will not be delivered to the new incarnation, but that was the case of the previous incarnation of the actor will not be delivered to the new incarnation, but that was the case
for remote references. The reason was that the target actor was looked up by its path on every message for remote references. The reason was that the target actor was looked up by its path on every message
delivery and the path didn't distinguish between the two incarnations of the actor. This has been fixed, and delivery and the path didn't distinguish between the two incarnations of the actor. This has been fixed, and
sending messages to remote references that points to a terminated actor will not be delivered to a new sending messages to remote references that points to a terminated actor will not be delivered to a new
actor with the same path. actor with the same path.
Equality of ``ActorRef`` has been changed to match the intention that an ``ActorRef`` corresponds to the target Equality of ``ActorRef`` has been changed to match the intention that an ``ActorRef`` corresponds to the target
actor instance. Two actor references are compared equal when they have the same path and point to the same actor instance. Two actor references are compared equal when they have the same path and point to the same
actor incarnation. A reference pointing to a terminated actor does not compare equal to a reference pointing actor incarnation. A reference pointing to a terminated actor does not compare equal to a reference pointing
to another (re-created) actor with the same path. Note that a restart of an actor caused by a failure still to another (re-created) actor with the same path. Note that a restart of an actor caused by a failure still
means that it's the same actor incarnation, i.e. a restart is not visible for the consumer of the ``ActorRef``. means that it's the same actor incarnation, i.e. a restart is not visible for the consumer of the ``ActorRef``.
Equality in 2.1 was only based on the path of the ``ActorRef``. If you need to keep track of actor references Equality in 2.1 was only based on the path of the ``ActorRef``. If you need to keep track of actor references
@ -175,8 +202,8 @@ in a collection and do not care about the exact actor incarnation you can use th
the identifier of the target actor is not taken into account when comparing actor paths. the identifier of the target actor is not taken into account when comparing actor paths.
Remote actor references acquired with ``actorFor`` do not include the full information about the underlying actor Remote actor references acquired with ``actorFor`` do not include the full information about the underlying actor
identity and therefore such references do not compare equal to references acquired with ``actorOf``, identity and therefore such references do not compare equal to references acquired with ``actorOf``,
``sender``, or ``context.self``. Because of this ``actorFor`` is deprecated, as explained in ``sender``, or ``context.self``. Because of this ``actorFor`` is deprecated, as explained in
:ref:`migration_2.2_actorSelection`. :ref:`migration_2.2_actorSelection`.
Note that when a parent actor is restarted its children are by default stopped and re-created, i.e. the child Note that when a parent actor is restarted its children are by default stopped and re-created, i.e. the child
@ -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 This may also have implications if you compare the ``ActorRef`` received in a ``Terminated`` message
with an expected ``ActorRef``. 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 Since Akka enforces parental supervision every actor is supervised and
(potentially) the supervisor of its children, it is advisable that you (potentially) the supervisor of its children, it is advisable that you
familiarize yourself with :ref:`actor-systems` and :ref:`supervision` and it familiarize yourself with :ref:`actor-systems` and :ref:`supervision` and it
may also help to read :ref:`actorOf-vs-actorFor` (the whole of may also help to read :ref:`addressing`.
:ref:`addressing` is recommended reading in any case).
Defining an Actor class Defining an Actor class
----------------------- -----------------------
@ -220,7 +219,7 @@ detects if the runtime class of the statically given actor subtype extends the
:class:`Stash` trait (this is a complicated way of saying that ``new Act with :class:`Stash` trait (this is a complicated way of saying that ``new Act with
Stash`` would not work because its runtime erased type is just an anonymous Stash`` would not work because its runtime erased type is just an anonymous
subtype of ``Act``). The purpose is to automatically use a dispatcher with the subtype of ``Act``). The purpose is to automatically use a dispatcher with the
appropriate deque-based mailbox, ``akka.actor.default-stash-dispatcher``. appropriate deque-based mailbox, ``akka.actor.default-stash-dispatcher``.
If you want to use this magic, simply extend :class:`ActWithStash`: If you want to use this magic, simply extend :class:`ActWithStash`:
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#act-with-stash .. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#act-with-stash
@ -340,7 +339,7 @@ mentioned above:
in turn by its supervisor, or if an actor is restarted due to a siblings in turn by its supervisor, or if an actor is restarted due to a siblings
failure. If the message is available, then that messages sender is also failure. If the message is available, then that messages sender is also
accessible in the usual way (i.e. by calling ``sender``). accessible in the usual way (i.e. by calling ``sender``).
This method is the best place for cleaning up, preparing hand-over to the This method is the best place for cleaning up, preparing hand-over to the
fresh actor instance, etc. By default it stops all children and calls fresh actor instance, etc. By default it stops all children and calls
:meth:`postStop`. :meth:`postStop`.
@ -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 sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
:obj:`ActorSystem`. :obj:`ActorSystem`.
Identifying Actors .. _actorSelection-scala:
==================
Identifying Actors via Actor Selection
======================================
As described in :ref:`addressing`, each actor has a unique logical path, which 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 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 are used by the system to look up actors, e.g. when a remote message is
received and the recipient is searched, but they are also useful more directly: received and the recipient is searched, but they are also useful more directly:
actors may look up other actors by specifying absolute or relative actors may look up other actors by specifying absolute or relative
paths—logical or physical—and receive back an :class:`ActorRef` with the paths—logical or physical—and receive back an :class:`ActorSelection` with the
result:: result::
context.actorFor("/user/serviceA/aggregator") // will look up this absolute path // will look up this absolute path
context.actorFor("../joe") // will look up sibling beneath same supervisor context.actorSelection("/user/serviceA/aggregator")
// will look up sibling beneath same supervisor
context.actorSelection("../joe")
The supplied path is parsed as a :class:`java.net.URI`, which basically means The supplied path is parsed as a :class:`java.net.URI`, which basically means
that it is split on ``/`` into path elements. If the path starts with ``/``, it that it is split on ``/`` into path elements. If the path starts with ``/``, it
@ -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 It should be noted that the ``..`` in actor paths here always means the logical
structure, i.e. the supervisor. structure, i.e. the supervisor.
If the path being looked up does not exist, a special actor reference is The path elements of an actor selection may contain wildcard patterns allowing for
returned which behaves like the actor systems dead letter queue but retains broadcasting of messages to that section::
its identity (i.e. the path which was looked up).
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 To acquire an :class:`ActorRef` for an :class:`ActorSelection` you need to
will have to send to it and await a reply in order to verify that ``serviceB`` send a message to the selection and use the ``sender`` reference of the reply from
is actually reachable and running. An example demonstrating actor look-up is the actor. There is a built-in ``Identify`` message that all Actors will understand
given in :ref:`remote-lookup-sample-scala`. 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 Messages and immutability
========================= =========================
@ -718,7 +745,7 @@ The other way of using :meth:`become` does not replace but add to the top of
the behavior stack. In this case care must be taken to ensure that the number the behavior stack. In this case care must be taken to ensure that the number
of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones
in the long run, otherwise this amounts to a memory leak (which is why this in the long run, otherwise this amounts to a memory leak (which is why this
behavior is not the default). behavior is not the default).
.. includecode:: code/docs/actor/ActorDocSpec.scala#swapper .. includecode:: code/docs/actor/ActorDocSpec.scala#swapper

View file

@ -327,6 +327,37 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
expectMsg("finished") 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 { "using pattern gracefulStop" in {
val actorRef = system.actorOf(Props[MyActor]) val actorRef = system.actorOf(Props[MyActor])
//#gracefulStop //#gracefulStop
@ -386,9 +417,9 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
lastSender must be === actor lastSender must be === actor
actor ! me actor ! me
expectMsg("reply") expectMsg("reply")
lastSender must be === system.actorFor("/user") lastSender.path.elements.mkString("/", "/", "") must be === "/user"
expectMsg("reply") expectMsg("reply")
lastSender must be === system.actorFor("/user") lastSender.path.elements.mkString("/", "/", "") must be === "/user"
} }
"using ActorDSL outside of akka.actor package" in { "using ActorDSL outside of akka.actor package" in {

View file

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

View file

@ -10,13 +10,15 @@ import akka.routing.RoundRobinRouter
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorPath
object RouterViaProgramDocSpec { object RouterViaProgramDocSpec {
case class Message1(nbr: Int) case class Message1(nbr: Int)
case class Reply1(name: String, m: Message1)
class ExampleActor1 extends Actor { class ExampleActor1 extends Actor {
def receive = { 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) } 1 to 6 foreach { i router ! Message1(i) }
val received = receiveN(6, 5.seconds.dilated) val received = receiveN(6, 5.seconds.dilated)
1 to 6 foreach { i 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) 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 { "demonstrate serialization of ActorRefs" in {
val theActorRef: ActorRef = system.deadLetters val theActorRef: ActorRef = system.deadLetters
val theActorSystem: ActorSystem = system val extendedSystem: ExtendedActorSystem = system.asInstanceOf[ExtendedActorSystem]
//#actorref-serializer //#actorref-serializer
// Serialize // Serialize
@ -168,7 +168,7 @@ package docs.serialization {
// Deserialize // Deserialize
// (beneath fromBinary) // (beneath fromBinary)
val deserializedActorRef = theActorSystem actorFor identifier val deserializedActorRef = extendedSystem.provider.resolveActorRef(identifier)
// Then just use the ActorRef // Then just use the ActorRef
//#actorref-serializer //#actorref-serializer
@ -182,7 +182,7 @@ package docs.serialization {
} }
def serializeTo(ref: ActorRef, remote: Address): String = def serializeTo(ref: ActorRef, remote: Address): String =
ref.path.toSerializationFormatWithAddress(ExternalAddress(theActorSystem).addressFor(remote)) ref.path.toSerializationFormatWithAddress(ExternalAddress(extendedSystem).addressFor(remote))
//#external-address //#external-address
} }

View file

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

View file

@ -69,7 +69,7 @@ Types of Remote Interaction
Akka has two ways of using remoting: 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)`` * 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. 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 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> 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:: .. 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 .. includecode:: ../../../akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/LookupApplication.scala
:include: setup :include: setup
Requests which come in via ``doSomething`` will be sent to the client actor 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 which will use the actor reference that was identified earlier. Observe how the actor
system name using in ``actorFor`` matches the remote systems name, as do IP 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"`` and port number. Top-level actors are always created below the ``"/user"``
guardian, which supervises them. guardian, which supervises them.
@ -481,15 +488,15 @@ SSL
SSL can be used as the remote transport by adding ``akka.remote.netty.ssl`` SSL can be used as the remote transport by adding ``akka.remote.netty.ssl``
to the ``enabled-transport`` configuration section. See a description of the settings to the ``enabled-transport`` configuration section. See a description of the settings
in the :ref:`remoting-scala-configuration` section. in the :ref:`remoting-scala-configuration` section.
The SSL support is implemented with Java Secure Socket Extension, please consult the offical The SSL support is implemented with Java Secure Socket Extension, please consult the offical
`Java Secure Socket Extension documentation <http://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html>`_ `Java Secure Socket Extension documentation <http://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html>`_
and related resources for troubleshooting. and related resources for troubleshooting.
.. note:: .. note::
When using SHA1PRNG on Linux it's recommended specify ``-Djava.security.egd=file:/dev/./urandom`` as argument When using SHA1PRNG on Linux it's recommended specify ``-Djava.security.egd=file:/dev/./urandom`` as argument
to the JVM to prevent blocking. It is NOT as secure because it reuses the seed. to the JVM to prevent blocking. It is NOT as secure because it reuses the seed.
Use '/dev/./urandom', not '/dev/urandom' as that doesn't work according to Use '/dev/./urandom', not '/dev/urandom' as that doesn't work according to
`Bug ID: 6202721 <http://bugs.sun.com/view_bug.do?bug_id=6202721>`_. `Bug ID: 6202721 <http://bugs.sun.com/view_bug.do?bug_id=6202721>`_.

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. 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:: .. note::
@ -213,7 +213,7 @@ This is usable if you want to communicate remotely with TypedActors on other mac
Lookup & Remoting 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 .. 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, 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 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 not bear any type information concerning the actors channels. Thus, when
looking up an actor with ``system.actorFor(...)`` you will only get an untyped looking up an actor with ``system.actorSelection(...)`` followed by an ``Identify``
:class:`ActorRef` and not a channel reference. This :class:`ActorRef` can of request you will only get an untyped :class:`ActorRef` and not a channel reference.
course manually be wrapped in a channel reference bearing the desired channels, This :class:`ActorRef` can of course manually be wrapped in a channel reference
but this is not a type-safe operation. 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 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 “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 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 durableMessage = RemoteMessageProtocol.parseFrom(bytes)
val message = MessageSerializer.deserialize(system, durableMessage.getMessage) 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: * 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) def node(role: RoleName): ActorPath = RootActorPath(testConductor.getAddressFor(role).await)

View file

@ -7,8 +7,10 @@ import akka.actor.Actor
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.Props import akka.actor.Props
import akka.pattern.ask import akka.pattern.ask
import testkit.{STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec} import testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec }
import akka.testkit._ import akka.testkit._
import akka.actor.Identify
import akka.actor.ActorIdentity
object LookupRemoteActorMultiJvmSpec extends MultiNodeConfig { object LookupRemoteActorMultiJvmSpec extends MultiNodeConfig {
@ -41,7 +43,10 @@ class LookupRemoteActorSpec extends MultiNodeSpec(LookupRemoteActorMultiJvmSpec)
"Remoting" must { "Remoting" must {
"lookup remote actor" taggedAs LongRunningTest in { "lookup remote actor" taggedAs LongRunningTest in {
runOn(slave) { 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) hello.isInstanceOf[RemoteActorRef] must be(true)
val masterAddress = testConductor.getAddressFor(master).await val masterAddress = testConductor.getAddressFor(master).await
(hello ? "identify").await.asInstanceOf[ActorRef].path.address must equal(masterAddress) (hello ? "identify").await.asInstanceOf[ActorRef].path.address must equal(masterAddress)

View file

@ -4,7 +4,6 @@
package akka.remote.testconductor package akka.remote.testconductor
import language.postfixOps import language.postfixOps
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.Props import akka.actor.Props
import akka.actor.Actor import akka.actor.Actor
@ -17,6 +16,8 @@ import java.net.InetSocketAddress
import java.net.InetAddress import java.net.InetAddress
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec, MultiNodeConfig } import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec, MultiNodeConfig }
import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.actor.Identify
import akka.actor.ActorIdentity
object TestConductorMultiJvmSpec extends MultiNodeConfig { object TestConductorMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false)) commonConfig(debugConfig(on = false))
@ -36,7 +37,10 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with ST
def initialParticipants = 2 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 { "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") if (UntrustedMode) log.debug("dropping daemon message in untrusted mode")
else { else {
if (LogReceive) log.debug("received daemon message {}", msgLog) if (LogReceive) log.debug("received daemon message {}", msgLog)
payload match { remoteDaemon ! payload
case m @ (_: DaemonMsg | _: Terminated)
remoteDaemon ! m
case x log.debug("remoteDaemon received illegal message {} from {}", x, sender)
}
} }
case l @ (_: LocalRef | _: RepointableRef) if l.isLocal case l @ (_: LocalRef | _: RepointableRef) if l.isLocal

View file

@ -28,9 +28,8 @@ private[akka] object RemoteActorRefProvider {
case object WaitTransportShutdown extends TerminatorState case object WaitTransportShutdown extends TerminatorState
case object Finished 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 import context.dispatcher
val systemGuardian = context.actorFor("/system")
startWith(Uninitialized, None) startWith(Uninitialized, None)
@ -79,10 +78,10 @@ private[akka] object RemoteActorRefProvider {
case _ super.!(message)(sender) 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)) // unwrap again in case the original message was DeadLetter(EndpointManager.Send(m))
case EndpointManager.Send(m, _, _) super.specialHandle(m) case EndpointManager.Send(m, _, _) super.specialHandle(m, sender)
case _ super.specialHandle(msg) case _ super.specialHandle(msg, sender)
} }
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
@ -148,7 +147,7 @@ private[akka] class RemoteActorRefProvider(
def init(system: ActorSystemImpl): Unit = { def init(system: ActorSystemImpl): Unit = {
local.init(system) local.init(system)
remotingTerminator = system.systemActorOf(Props[RemotingTerminator], "remoting-terminator") remotingTerminator = system.systemActorOf(Props(new RemotingTerminator(local.systemGuardian)), "remoting-terminator")
val internals = Internals( val internals = Internals(
remoteDaemon = { remoteDaemon = {
@ -244,7 +243,7 @@ private[akka] class RemoteActorRefProvider(
new RemoteActorRef(transport, localAddress, rpath, supervisor, Some(props), Some(d)) new RemoteActorRef(transport, localAddress, rpath, supervisor, Some(props), Some(d))
} catch { } catch {
case NonFatal(e) 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) 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 = { def actorFor(path: ActorPath): InternalActorRef = {
if (hasAddress(path.address)) actorFor(rootGuardian, path.elements) if (hasAddress(path.address)) actorFor(rootGuardian, path.elements)
else try { else try {
@ -261,11 +261,12 @@ private[akka] class RemoteActorRefProvider(
path, Nobody, props = None, deploy = None) path, Nobody, props = None, deploy = None)
} catch { } catch {
case NonFatal(e) 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) new EmptyLocalActorRef(this, path, eventStream)
} }
} }
@deprecated("use actorSelection instead of actorFor", "2.2")
def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match { def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
case ActorPathExtractor(address, elems) case ActorPathExtractor(address, elems)
if (hasAddress(address)) actorFor(rootGuardian, elems) if (hasAddress(address)) actorFor(rootGuardian, elems)
@ -276,40 +277,81 @@ private[akka] class RemoteActorRefProvider(
rootPath, Nobody, props = None, deploy = None) rootPath, Nobody, props = None, deploy = None)
} catch { } catch {
case NonFatal(e) 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) new EmptyLocalActorRef(this, rootPath, eventStream)
} }
} }
case _ local.actorFor(ref, path) 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 * INTERNAL API
* Called in deserialization of incoming remote messages. In this case the correct local address is known, therefore * Called in deserialization of incoming remote messages where the correct local address is known.
* this method is faster than the actorFor above.
*/ */
def actorForWithLocalAddress(ref: InternalActorRef, path: String, localAddress: Address): InternalActorRef = { private[akka] def resolveActorRefWithLocalAddress(path: String, localAddress: Address): InternalActorRef = {
path match { path match {
case ActorPathExtractor(address, elems) case ActorPathExtractor(address, elems)
if (hasAddress(address)) actorFor(rootGuardian, elems) if (hasAddress(address)) local.resolveActorRef(rootGuardian, elems)
else new RemoteActorRef(transport, localAddress, else
new RootActorPath(address) / elems, Nobody, props = None, deploy = None) new RemoteActorRef(transport, localAddress, RootActorPath(address) / elems,
Nobody, props = None, deploy = None)
case _ case _
local.actorFor(ref, path) log.debug("resolve of unknown path [{}] failed", path)
deadLetters
} }
} }
def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef = def resolveActorRef(path: String): ActorRef = path match {
local.actorFor(ref, path) 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. * Using (checking out) actor on a specific node.
*/ */
def useActorOnNode(path: ActorPath, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = { def useActorOnNode(ref: ActorRef, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, path) 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 // 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] = { def getExternalAddressFor(addr: Address): Option[Address] = {
@ -374,7 +416,7 @@ private[akka] class RemoteActorRef private[akka] (
override def provider: RemoteActorRefProvider = remote.provider override def provider: RemoteActorRefProvider = remote.provider
def start(): Unit = 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()) def suspend(): Unit = sendSystemMessage(Suspend())

View file

@ -13,6 +13,11 @@ import akka.actor.ActorRefWithCell
import akka.actor.ActorRefScope import akka.actor.ActorRefScope
import akka.util.Switch import akka.util.Switch
import akka.actor.RootActorPath 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 * 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 case Terminated(child: ActorRefWithCell) if child.asInstanceOf[ActorRefScope].isLocal
terminating.locked { terminating.locked {
removeChild(child.path.elements.drop(1).mkString("/")) removeChild(child.path.elements.drop(1).mkString("/"))

View file

@ -6,7 +6,7 @@ package akka.remote.serialization
import akka.serialization.{ Serializer, Serialization } import akka.serialization.{ Serializer, Serialization }
import com.google.protobuf.Message import com.google.protobuf.Message
import akka.actor.{ ActorSystem, ActorRef } import akka.actor.{ ActorSystem, ActorRef, ExtendedActorSystem }
import akka.remote.RemoteProtocol.ActorRefProtocol import akka.remote.RemoteProtocol.ActorRefProtocol
object ProtobufSerializer { object ProtobufSerializer {
@ -24,8 +24,8 @@ object ProtobufSerializer {
* from Akka's protobuf representation in the supplied * from Akka's protobuf representation in the supplied
* [[akka.actor.ActorSystem]]. * [[akka.actor.ActorSystem]].
*/ */
def deserializeActorRef(system: ActorSystem, refProtocol: ActorRefProtocol): ActorRef = def deserializeActorRef(system: ExtendedActorSystem, refProtocol: ActorRefProtocol): ActorRef =
system.actorFor(refProtocol.getPath) 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. // Write once variable initialized when Listen is called.
@volatile protected var manager: ActorRef = _ @volatile protected var manager: ActorRef = _
// FIXME #3074 how to replace actorFor here?
private def registerManager(): Future[ActorRef] = private def registerManager(): Future[ActorRef] =
(system.actorFor("/system/transports") ? RegisterTransportActor(managerProps, managerName)).mapTo[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 = { localAddress: Address): Message = {
val msgPdu = RemoteMessageProtocol.parseFrom(raw.toArray) val msgPdu = RemoteMessageProtocol.parseFrom(raw.toArray)
Message( Message(
recipient = provider.actorForWithLocalAddress(provider.rootGuardian, msgPdu.getRecipient.getPath, localAddress), recipient = provider.resolveActorRefWithLocalAddress(msgPdu.getRecipient.getPath, localAddress),
recipientAddress = AddressFromURIString(msgPdu.getRecipient.getPath), recipientAddress = AddressFromURIString(msgPdu.getRecipient.getPath),
serializedMessage = msgPdu.getMessage, serializedMessage = msgPdu.getMessage,
senderOption = if (!msgPdu.hasSender) None 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 = { private def decodeControlPdu(controlPdu: RemoteControlProtocol): AkkaPdu = {

View file

@ -49,4 +49,10 @@ akka {
expectMsg(60.seconds, path) 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._ import scala.concurrent.duration._
object RemotingSpec { object RemotingSpec {
case class ActorForReq(s: String)
case class ActorSelReq(s: String)
class Echo1 extends Actor { class Echo1 extends Actor {
var target: ActorRef = context.system.deadLetters var target: ActorRef = context.system.deadLetters
def receive = { def receive = {
case (p: Props, n: String) sender ! context.actorOf(Props[Echo1], n) case (p: Props, n: String) sender ! context.actorOf(Props[Echo1], n)
case ex: Exception throw ex 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 case x target = sender; sender ! x
} }
@ -110,8 +115,10 @@ object RemotingSpec {
actor.deployment { actor.deployment {
/blub.remote = "akka.test://remote-sys@localhost:12346" /blub.remote = "akka.test://remote-sys@localhost:12346"
/looker/child.remote = "akka.test://remote-sys@localhost:12346" /looker1/child.remote = "akka.test://remote-sys@localhost:12346"
/looker/child/grandchild.remote = "akka.test://RemotingSpec@localhost:12345" /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.*"))) EventFilter.warning(pattern = "received dead letter.*")))
sys.actorOf(Props[Echo2], name = "echo") sys.actorOf(Props[Echo2], name = "echo")
} }
val moreRefs = moreSystems map (sys system.actorFor(RootActorPath(addr(sys, "tcp")) / "user" / "echo")) val moreRefs = moreSystems map (sys system.actorSelection(RootActorPath(addr(sys, "tcp")) / "user" / "echo"))
val aliveEcho = system.actorFor(RootActorPath(addr(remoteSystem, "tcp")) / "user" / "echo") val aliveEcho = system.actorSelection(RootActorPath(addr(remoteSystem, "tcp")) / "user" / "echo")
val n = 100 val n = 100
// first everything is up and running // 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 system.actorFor("akka.test://remote-sys@localhost:12346/user/otherEcho1") ! 76
expectMsg(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 { "look-up actors across node boundaries" in {
val l = system.actorOf(Props(new Actor { val l = system.actorOf(Props(new Actor {
def receive = { def receive = {
case (p: Props, n: String) sender ! context.actorOf(p, n) 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) // child is configured to be deployed on remote-sys (remoteSystem)
l ! ((Props[Echo1], "child")) l ! ((Props[Echo1], "child"))
val child = expectMsgType[ActorRef] 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.asInstanceOf[ActorRefScope].isLocal must be(true)
grandchild ! 43 grandchild ! 43
expectMsg(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.isInstanceOf[RemoteActorRef] must be(true)
myref ! 44 myref ! 44
expectMsg(44) expectMsg(44)
lastSender must be(grandchild) lastSender must be(grandchild)
lastSender must be theSameInstanceAs grandchild lastSender must be theSameInstanceAs grandchild
child.asInstanceOf[RemoteActorRef].getParent must be(l) child.asInstanceOf[RemoteActorRef].getParent must be(l)
system.actorFor("/user/looker/child") must be theSameInstanceAs child system.actorFor("/user/looker1/child") must be theSameInstanceAs child
Await.result(l ? "child/..", timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l Await.result(l ? ActorForReq("child/.."), timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
Await.result(system.actorFor(system / "looker" / "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) watch(child)
child ! PoisonPill 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) child2.path.uid must not be (child.path.uid)
child ! 46 child ! 46
expectNoMsg(1.second) expectNoMsg(1.second)
system.actorFor(system / "looker" / "child") ! 47 system.actorFor(system / "looker1" / "child") ! 47
expectMsg(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 { "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 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 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") val proxySsl = system.actorOf(Props(new Proxy(remoteEchoHereSsl, testActor)), "proxy-ssl")
proxySsl ! otherGuy EventFilter[RemoteTransportException](start = "Error while resolving address", occurrences = 1).intercept {
expectMsg(3.seconds, ("pong", otherGuyRemoteTest)) proxySsl ! otherGuy
expectMsg(3.seconds, ("pong", otherGuyRemoteTest))
}(otherSystem)
} finally { } finally {
otherSystem.shutdown() otherSystem.shutdown()
otherSystem.awaitTermination(5.seconds.dilated) 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 val otherAddress = other.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.defaultAddress
"support tell" in { "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) here ! (("ping", i))
for (i 1 to 1000) expectMsgPF(timeout.duration) { case (("pong", i), `testActor`) true } 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 { "support ask" in {
import system.dispatcher 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)] 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")) 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 addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
val rootB = RootActorPath(addressB) 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 { "AkkaProtocolTransport" must {
"guarantee at-most-once delivery and message ordering despite packet loss" taggedAs TimingTest in { "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 remote = systemB.actorOf(Props[Echo], "echo")
val rootB = RootActorPath(systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress) 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 = { def throttle(direction: Direction, mode: ThrottleMode): Boolean = {
val rootBAddress = Address("akka", "systemB", "localhost", rootB.address.port.get) val rootBAddress = Address("akka", "systemB", "localhost", rootB.address.port.get)

View file

@ -1,21 +1,16 @@
package sample.cluster.stats.japi; package sample.cluster.stats.japi;
import scala.concurrent.Future;
import sample.cluster.stats.japi.StatsMessages.JobFailed; import sample.cluster.stats.japi.StatsMessages.JobFailed;
import sample.cluster.stats.japi.StatsMessages.StatsJob; import sample.cluster.stats.japi.StatsMessages.StatsJob;
import akka.actor.ActorRef; import akka.actor.ActorSelection;
import akka.actor.Address; import akka.actor.Address;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.dispatch.Recover;
import akka.cluster.Cluster; import akka.cluster.Cluster;
import akka.cluster.ClusterEvent.CurrentClusterState; import akka.cluster.ClusterEvent.CurrentClusterState;
import akka.cluster.ClusterEvent.RoleLeaderChanged; import akka.cluster.ClusterEvent.RoleLeaderChanged;
import akka.event.Logging; import akka.event.Logging;
import akka.event.LoggingAdapter; 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 //#facade
public class StatsFacade extends UntypedActor { public class StatsFacade extends UntypedActor {
@ -23,7 +18,7 @@ public class StatsFacade extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this); LoggingAdapter log = Logging.getLogger(getContext().system(), this);
Cluster cluster = Cluster.get(getContext().system()); Cluster cluster = Cluster.get(getContext().system());
Address currentMaster = null; ActorSelection currentMaster = null;
//subscribe to cluster changes, RoleLeaderChanged //subscribe to cluster changes, RoleLeaderChanged
@Override @Override
@ -44,30 +39,29 @@ public class StatsFacade extends UntypedActor {
getSelf()); getSelf());
} else if (message instanceof StatsJob) { } else if (message instanceof StatsJob) {
StatsJob job = (StatsJob) message; currentMaster.tell(message, getSender());
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());
} else if (message instanceof CurrentClusterState) { } else if (message instanceof CurrentClusterState) {
CurrentClusterState state = (CurrentClusterState) message; CurrentClusterState state = (CurrentClusterState) message;
currentMaster = state.getRoleLeader("compute"); setCurrentMaster(state.getRoleLeader("compute"));
} else if (message instanceof RoleLeaderChanged) { } else if (message instanceof RoleLeaderChanged) {
RoleLeaderChanged leaderChanged = (RoleLeaderChanged) message; RoleLeaderChanged leaderChanged = (RoleLeaderChanged) message;
if (leaderChanged.role().equals("compute")) if (leaderChanged.role().equals("compute"))
currentMaster = leaderChanged.getLeader(); setCurrentMaster(leaderChanged.getLeader());
} else { } else {
unhandled(message); unhandled(message);
} }
} }
void setCurrentMaster(Address address) {
if (address == null)
currentMaster = null;
else
currentMaster = getContext().actorSelection(address +
"/user/singleton/statsService");
}
} }
//#facade //#facade

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -115,17 +115,12 @@ abstract class TransformationSampleSpec extends MultiNodeSpec(TransformationSamp
} }
def assertServiceOk(): Unit = { 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, // eventually the service should be ok,
// backends might not have registered initially // backends might not have registered initially
awaitCond { awaitAssert {
transformationFrontend ! TransformationJob("hello") transformationFrontend ! TransformationJob("hello")
expectMsgPF() { expectMsgType[TransformationResult](1.second).text must be("HELLO")
case unavailble: JobFailed false
case TransformationResult(result)
result must be("HELLO")
true
}
} }
} }

View file

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

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: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] [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 Started Lookup Application
Not ready yet
Not ready yet
Add result: 0 + 22 = 22 Add result: 0 + 22 = 22
Add result: 41 + 71 = 112 Add result: 41 + 71 = 112
Add result: 61 + 14 = 75 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; package sample.remote.calculator.java;
import akka.actor.ActorRef;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import java.text.DecimalFormat;
import java.text.NumberFormat;
//#actor //#actor
public class JCreationActor extends UntypedActor { 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 @Override
public void onReceive(Object message) throws Exception { public void onReceive(Object message) throws Exception {
if (message instanceof InternalMsg.MathOpMsg) { if (message instanceof Op.MathOp) {
// forward math op to server actor // send message to server actor
InternalMsg.MathOpMsg msg = (InternalMsg.MathOpMsg) message; remoteActor.tell(message, getSelf());
msg.getActor().tell(msg.getMathOp(), getSelf());
} else if (message instanceof Op.MathResult) { } else if (message instanceof Op.MultiplicationResult) {
Op.MultiplicationResult result = (Op.MultiplicationResult) message;
System.out.printf("Mul result: %d * %d = %d\n",
result.getN1(), result.getN2(), result.getResult());
// receive reply from server actor } else if (message instanceof Op.DivisionResult) {
Op.DivisionResult result = (Op.DivisionResult) message;
System.out.printf("Div result: %.0f / %d = %.2f\n",
result.getN1(), result.getN2(), result.getResult());
if (message instanceof Op.MultiplicationResult) {
Op.MultiplicationResult result = (Op.MultiplicationResult) message;
System.out.println("Mul result: " + 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()));
}
} else { } else {
unhandled(message); unhandled(message);
} }

View file

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

View file

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

View file

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

View file

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

View file

@ -16,12 +16,13 @@ class CreationApplication extends Bootable {
//#setup //#setup
val system = val system =
ActorSystem("RemoteCreation", ConfigFactory.load.getConfig("remotecreation")) ActorSystem("RemoteCreation", ConfigFactory.load.getConfig("remotecreation"))
val localActor = system.actorOf(Props[CreationActor], "creationActor") val remoteActor = system.actorOf(Props[AdvancedCalculatorActor],
val remoteActor = name = "advancedCalculator")
system.actorOf(Props[AdvancedCalculatorActor], "advancedCalculator") val localActor = system.actorOf(Props(new CreationActor(remoteActor)),
name = "creationActor")
def doSomething(op: MathOp): Unit = def doSomething(op: MathOp): Unit =
localActor ! ((remoteActor, op)) localActor ! op
//#setup //#setup
def startup() { def startup() {
@ -33,14 +34,14 @@ class CreationApplication extends Bootable {
} }
//#actor //#actor
class CreationActor extends Actor { class CreationActor(remoteActor: ActorRef) extends Actor {
def receive = { def receive = {
case (actor: ActorRef, op: MathOp) actor ! op case op: MathOp remoteActor ! op
case result: MathResult result match { case result: MathResult result match {
case MultiplicationResult(n1, n2, r) 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) 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 * comments like //#<tag> are there for inclusion into docs, please dont remove
*/ */
import akka.kernel.Bootable
import scala.util.Random import scala.util.Random
//#imports import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.{ ActorRef, Props, Actor, ActorSystem } import akka.actor.{ ActorRef, Props, Actor, ActorSystem }
import akka.actor.Identify
import akka.actor.ActorIdentity
import akka.kernel.Bootable
import akka.actor.ReceiveTimeout
//#imports //#imports
class LookupApplication extends Bootable { class LookupApplication extends Bootable {
//#setup //#setup
val system = val system =
ActorSystem("LookupApplication", ConfigFactory.load.getConfig("remotelookup")) ActorSystem("LookupApplication", ConfigFactory.load.getConfig("remotelookup"))
val actor = system.actorOf(Props[LookupActor], "lookupActor") val remotePath =
val remoteActor = system.actorFor( "akka.tcp://CalculatorApplication@127.0.0.1:2552/user/simpleCalculator"
"akka.tcp://CalculatorApplication@127.0.0.1:2552/user/simpleCalculator") val actor = system.actorOf(Props(new LookupActor(remotePath)), "lookupActor")
def doSomething(op: MathOp): Unit = def doSomething(op: MathOp): Unit =
actor ! ((remoteActor, op)) actor ! op
//#setup //#setup
def startup() { def startup() {
@ -35,14 +38,30 @@ class LookupApplication extends Bootable {
} }
//#actor //#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 = { 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 result: MathResult result match {
case AddResult(n1, n2, r) 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) 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) val latch = new TestLatch(1)(system)
system.registerOnTermination(latch.countDown()) system.registerOnTermination(latch.countDown())
system.actorFor("/") ! PoisonPill system.actorSelection("/") ! PoisonPill
Await.ready(latch, 2 seconds) Await.ready(latch, 2 seconds)
} }

View file

@ -87,7 +87,11 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout {
"watch actors when queue non-empty" in { "watch actors when queue non-empty" in {
val probe = TestProbe() 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.ref ! "hello"
probe watch target probe watch target
probe.expectMsg(1.seconds, "hello") probe.expectMsg(1.seconds, "hello")