Remove remainings of actorFor, #26190

* remove undefinedUid specials related to actorFor
* mention in migration guide
* remove special case for watch of undefinedUid
This commit is contained in:
Patrik Nordwall 2019-04-29 15:46:42 +02:00
parent 267d96ecf1
commit 14de568570
19 changed files with 69 additions and 743 deletions

View file

@ -1,294 +0,0 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor
import language.postfixOps
import akka.testkit._
import scala.concurrent.duration._
import scala.concurrent.Await
import akka.pattern.ask
import com.github.ghik.silencer.silent
object ActorLookupSpec {
final case class Create(child: String)
trait Query
final case class LookupElems(path: Iterable[String]) extends Query
final case class LookupString(path: String) extends Query
final case class LookupPath(path: ActorPath) extends Query
final case class GetSender(to: ActorRef) extends Query
val p = Props[Node]
@silent
class Node extends Actor {
def receive = {
case Create(name) => sender() ! context.actorOf(p, name)
case LookupElems(path) => sender() ! context.actorFor(path)
case LookupString(path) => sender() ! context.actorFor(path)
case LookupPath(path) => sender() ! context.actorFor(path)
case GetSender(ref) => ref ! sender()
}
}
}
@silent
class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
import ActorLookupSpec._
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) => system.actorFor("/").path / elems
}, system.eventStream)
"An ActorSystem" must {
"find actors by looking up their path" in {
system.actorFor(c1.path) should ===(c1)
system.actorFor(c2.path) should ===(c2)
system.actorFor(c21.path) should ===(c21)
system.actorFor(system / "c1") should ===(c1)
system.actorFor(system / "c2") should ===(c2)
system.actorFor(system / "c2" / "c21") should ===(c21)
system.actorFor(system.child("c2").child("c21")) should ===(c21) // test Java API
system.actorFor(system / Seq("c2", "c21")) should ===(c21)
import scala.collection.JavaConverters._
system.actorFor(system.descendant(Seq("c2", "c21").asJava)) // test Java API
}
"find actors by looking up their string representation" in {
// this is only true for local actor references
system.actorFor(c1.path.toString) should ===(c1)
system.actorFor(c2.path.toString) should ===(c2)
system.actorFor(c21.path.toString) should ===(c21)
}
"take actor incarnation into account when comparing actor references" in {
val name = "abcdefg"
val a1 = system.actorOf(p, name)
watch(a1)
a1 ! PoisonPill
expectTerminated(a1)
// let it be completely removed from user guardian
expectNoMessage(1 second)
// not equal because it's terminated
system.actorFor(a1.path.toString) should not be (a1)
val a2 = system.actorOf(p, name)
a2.path should ===(a1.path)
a2.path.toString should ===(a1.path.toString)
a2 should not be (a1)
a2.toString should not be (a1.toString)
watch(a2)
a2 ! PoisonPill
expectTerminated(a2)
}
"find actors by looking up their root-anchored relative path" in {
system.actorFor(c1.path.toStringWithoutAddress) should ===(c1)
system.actorFor(c2.path.toStringWithoutAddress) should ===(c2)
system.actorFor(c21.path.toStringWithoutAddress) should ===(c21)
}
"find actors by looking up their relative path" in {
system.actorFor(c1.path.elements.mkString("/")) should ===(c1)
system.actorFor(c2.path.elements.mkString("/")) should ===(c2)
system.actorFor(c21.path.elements.mkString("/")) should ===(c21)
}
"find actors by looking up their path elements" in {
system.actorFor(c1.path.elements) should ===(c1)
system.actorFor(c2.path.elements) should ===(c2)
system.actorFor(c21.path.getElements) should ===(c21) // test Java API
}
"find system-generated actors" in {
system.actorFor("/user") should ===(user)
system.actorFor("/deadLetters") should ===(system.deadLetters)
system.actorFor("/system") should ===(syst)
system.actorFor(syst.path) should ===(syst)
system.actorFor(syst.path.toString) should ===(syst)
system.actorFor("/") should ===(root)
system.actorFor("..") should ===(root)
system.actorFor(root.path) should ===(root)
system.actorFor(root.path.toString) should ===(root)
system.actorFor("user") should ===(user)
system.actorFor("deadLetters") should ===(system.deadLetters)
system.actorFor("system") should ===(syst)
system.actorFor("user/") should ===(user)
system.actorFor("deadLetters/") should ===(system.deadLetters)
system.actorFor("system/") should ===(syst)
}
"return deadLetters or EmptyLocalActorRef, respectively, for non-existing paths" in {
def check(lookup: ActorRef, result: ActorRef) = {
lookup.getClass should ===(result.getClass)
lookup should ===(result)
}
check(system.actorFor("a/b/c"), empty("a/b/c"))
check(system.actorFor(""), system.deadLetters)
check(system.actorFor("akka://all-systems/Nobody"), system.deadLetters)
check(system.actorFor("akka://all-systems/user"), system.deadLetters)
check(system.actorFor(system / "hallo"), empty("user/hallo"))
check(system.actorFor(Seq()), system.deadLetters)
check(system.actorFor(Seq("a")), empty("a"))
}
"find temporary actors" in {
val f = c1 ? GetSender(testActor)
val a = expectMsgType[ActorRef]
a.path.elements.head should ===("temp")
system.actorFor(a.path) should ===(a)
system.actorFor(a.path.toString) should ===(a)
system.actorFor(a.path.elements) should ===(a)
system.actorFor(a.path.toString + "/") should ===(a)
system.actorFor(a.path.toString + "/hallo").isTerminated should ===(true)
f.isCompleted should ===(false)
a.isTerminated should ===(false)
a ! 42
f.isCompleted should ===(true)
Await.result(f, timeout.duration) should ===(42)
// clean-up is run as onComplete callback, i.e. dispatched on another thread
awaitCond(system.actorFor(a.path).isTerminated, 1 second)
}
}
"An ActorContext" must {
val all = Seq(c1, c2, c21)
"find actors by looking up their path" in {
def check(looker: ActorRef, pathOf: ActorRef, result: ActorRef): Unit = {
Await.result(looker ? LookupPath(pathOf.path), timeout.duration) should ===(result)
}
for {
looker <- all
target <- all
} check(looker, target, target)
}
"find actors by looking up their string representation" in {
def check(looker: ActorRef, pathOf: ActorRef, result: ActorRef): Unit = {
Await.result(looker ? LookupString(pathOf.path.toString), timeout.duration) should ===(result)
// with uid
Await.result(looker ? LookupString(pathOf.path.toSerializationFormat), timeout.duration) should ===(result)
// with trailing /
Await.result(looker ? LookupString(pathOf.path.toString + "/"), timeout.duration) should ===(result)
}
for {
looker <- all
target <- all
} check(looker, target, target)
}
"find actors by looking up their root-anchored relative path" in {
def check(looker: ActorRef, pathOf: ActorRef, result: ActorRef): Unit = {
Await.result(looker ? LookupString(pathOf.path.toStringWithoutAddress), timeout.duration) should ===(result)
Await.result(looker ? LookupString(pathOf.path.elements.mkString("/", "/", "/")), timeout.duration) should ===(
result)
}
for {
looker <- all
target <- all
} check(looker, target, target)
}
"find actors by looking up their relative path" in {
def check(looker: ActorRef, result: ActorRef, elems: String*): Unit = {
Await.result(looker ? LookupElems(elems), timeout.duration) should ===(result)
Await.result(looker ? LookupString(elems.mkString("/")), timeout.duration) should ===(result)
Await.result(looker ? LookupString(elems.mkString("", "/", "/")), timeout.duration) should ===(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): Unit = {
for (looker <- all) {
Await.result(looker ? LookupPath(target.path), timeout.duration) should ===(target)
Await.result(looker ? LookupString(target.path.toString), timeout.duration) should ===(target)
Await.result(looker ? LookupString(target.path.toString + "/"), timeout.duration) should ===(target)
Await.result(looker ? LookupString(target.path.toStringWithoutAddress), timeout.duration) should ===(target)
if (target != root)
Await.result(looker ? LookupString(target.path.elements.mkString("/", "/", "/")), timeout.duration) should ===(
target)
}
}
for (target <- Seq(root, syst, user, system.deadLetters)) check(target)
}
"return deadLetters or EmptyLocalActorRef, respectively, for non-existing paths" in {
import scala.collection.JavaConverters._
def checkOne(looker: ActorRef, query: Query, result: ActorRef): Unit = {
val lookup = Await.result(looker ? query, timeout.duration)
lookup.getClass should be(result.getClass)
lookup should ===(result)
}
def check(looker: ActorRef): Unit = {
val lookname = looker.path.elements.mkString("", "/", "/")
for ((l, r) <- Seq(
LookupString("a/b/c") -> empty(lookname + "a/b/c"),
LookupString("") -> system.deadLetters,
LookupString("akka://all-systems/Nobody") -> system.deadLetters,
LookupPath(system / "hallo") -> empty("user/hallo"),
LookupPath(looker.path.child("hallo")) -> empty(lookname + "hallo"), // test Java API
LookupPath(looker.path.descendant(Seq("a", "b").asJava)) -> empty(lookname + "a/b"), // test Java API
LookupElems(Seq()) -> system.deadLetters,
LookupElems(Seq("a")) -> empty(lookname + "a"))) checkOne(looker, l, r)
}
for (looker <- all) check(looker)
}
"find temporary actors" in {
val f = c1 ? GetSender(testActor)
val a = expectMsgType[ActorRef]
a.path.elements.head should ===("temp")
Await.result(c2 ? LookupPath(a.path), timeout.duration) should ===(a)
Await.result(c2 ? LookupString(a.path.toString), timeout.duration) should ===(a)
Await.result(c2 ? LookupString(a.path.toStringWithoutAddress), timeout.duration) should ===(a)
Await.result(c2 ? LookupString("../../" + a.path.elements.mkString("/")), timeout.duration) should ===(a)
Await.result(c2 ? LookupString(a.path.toString + "/"), timeout.duration) should ===(a)
Await.result(c2 ? LookupString(a.path.toStringWithoutAddress + "/"), timeout.duration) should ===(a)
Await.result(c2 ? LookupString("../../" + a.path.elements.mkString("/") + "/"), timeout.duration) should ===(a)
Await.result(c2 ? LookupElems(Seq("..", "..") ++ a.path.elements), timeout.duration) should ===(a)
Await.result(c2 ? LookupElems(Seq("..", "..") ++ a.path.elements :+ ""), timeout.duration) should ===(a)
f.isCompleted should ===(false)
a.isTerminated should ===(false)
a ! 42
f.isCompleted should ===(true)
Await.result(f, timeout.duration) should ===(42)
// clean-up is run as onComplete callback, i.e. dispatched on another thread
awaitCond(Await.result(c2 ? LookupPath(a.path), timeout.duration).asInstanceOf[ActorRef].isTerminated, 1 second)
}
}
}

View file

@ -68,8 +68,7 @@ object ActorSystemSpec {
@silent @silent
final case class FastActor(latch: TestLatch, testActor: ActorRef) extends Actor { final case class FastActor(latch: TestLatch, testActor: ActorRef) extends Actor {
val ref1 = context.actorOf(Props.empty) val ref1 = context.actorOf(Props.empty)
val ref2 = context.actorFor(ref1.path.toString) context.actorSelection(ref1.path.toString).tell(Identify(ref1), testActor)
testActor ! ref2.getClass
latch.countDown() latch.countDown()
def receive = { def receive = {
@ -293,7 +292,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
"find actors that just have been created" in { "find actors that just have been created" in {
system.actorOf(Props(new FastActor(TestLatch(), testActor)).withDispatcher("slow")) system.actorOf(Props(new FastActor(TestLatch(), testActor)).withDispatcher("slow"))
expectMsgType[Class[_]] should ===(classOf[LocalActorRef]) expectMsgType[ActorIdentity].ref should !==(None)
} }
"reliable deny creation of actors while shutting down" in { "reliable deny creation of actors while shutting down" in {

View file

@ -37,20 +37,16 @@ object LocalActorRefProviderSpec {
class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.config) { class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.config) {
"An LocalActorRefProvider" must { "An LocalActorRefProvider" must {
"find actor refs using actorFor" in { "find child actor with URL encoded name" in {
val a = system.actorOf(Props(new Actor { def receive = { case _ => } }))
val b = system.actorFor(a.path)
a should ===(b)
}
"find child actor with URL encoded name using actorFor" in {
val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552" val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552"
val a = system.actorOf(Props(new Actor { val a = system.actorOf(Props(new Actor {
val child = context.actorOf(Props.empty, name = childName) val child = context.actorOf(Props.empty, name = childName)
def receive = { def receive = {
case "lookup" => case "lookup" =>
if (childName == child.path.name) sender() ! context.actorFor(childName) if (childName == child.path.name) {
else sender() ! s"$childName is not ${child.path.name}!" val resolved = system.asInstanceOf[ExtendedActorSystem].provider.resolveActorRef(child.path)
sender() ! resolved
} else sender() ! s"$childName is not ${child.path.name}!"
} }
})) }))
a.tell("lookup", testActor) a.tell("lookup", testActor)

View file

@ -26,8 +26,7 @@ class AskSpec extends AkkaSpec {
} }
"return broken promises on DeadLetters" in { "return broken promises on DeadLetters" in {
val dead = system.actorFor("/system/deadLetters") val f = system.deadLetters.ask(42)(1 second)
val f = dead.ask(42)(1 second)
f.isCompleted should ===(true) f.isCompleted should ===(true)
f.value.get match { f.value.get match {
case Failure(_: AskTimeoutException) => case Failure(_: AskTimeoutException) =>
@ -37,7 +36,7 @@ class AskSpec extends AkkaSpec {
"return broken promises on EmptyLocalActorRefs" in { "return broken promises on EmptyLocalActorRefs" in {
implicit val timeout = Timeout(5 seconds) implicit val timeout = Timeout(5 seconds)
val empty = system.actorFor("unknown") val empty = system.asInstanceOf[ExtendedActorSystem].provider.resolveActorRef("/user/unknown")
val f = empty ? 3.14 val f = empty ? 3.14
f.isCompleted should ===(true) f.isCompleted should ===(true)
f.value.get match { f.value.get match {

View file

@ -168,9 +168,9 @@ class ConfiguredLocalRoutingSpec
"not get confused when trying to wildcard-configure children" in { "not get confused when trying to wildcard-configure children" in {
system.actorOf(FromConfig.props(routeeProps = Props(classOf[SendRefAtStartup], testActor)), "weird") system.actorOf(FromConfig.props(routeeProps = Props(classOf[SendRefAtStartup], testActor)), "weird")
val recv = Set() ++ (for (_ <- 1 to 3) yield expectMsgType[ActorRef]) val recv = (for (_ <- 1 to 3) yield expectMsgType[ActorRef].path.elements.mkString("/", "/", "")).toSet
@silent @silent
val expc = Set('a', 'b', 'c').map(i => system.actorFor("/user/weird/$" + i)) val expc = Set('a', 'b', 'c').map(i => "/user/weird/$" + i)
recv should ===(expc) recv should ===(expc)
expectNoMessage(1 second) expectNoMessage(1 second)
} }

View file

@ -4,3 +4,12 @@ ProblemFilters.exclude[MissingClassProblem]("akka.actor.ActorDSL$")
ProblemFilters.exclude[MissingClassProblem]("akka.actor.ActorDSL") ProblemFilters.exclude[MissingClassProblem]("akka.actor.ActorDSL")
ProblemFilters.exclude[MissingClassProblem]("akka.actor.ActorDSL$*") ProblemFilters.exclude[MissingClassProblem]("akka.actor.ActorDSL$*")
ProblemFilters.exclude[MissingClassProblem]("akka.actor.dsl.*") ProblemFilters.exclude[MissingClassProblem]("akka.actor.dsl.*")
# #26190 remove actorFor
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorCell.actorFor")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorRefProvider.actorFor")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.LocalActorRefProvider.actorFor")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorRefFactory.actorFor")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.actorFor")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ChildActorPath.this")
ProblemFilters.exclude[MissingClassProblem]("akka.actor.dungeon.UndefinedUidActorRef")

View file

@ -420,7 +420,7 @@ private[akka] object ActorCell {
// Note that this uid is also used as hashCode in ActorRef, so be careful // Note that this uid is also used as hashCode in ActorRef, so be careful
// to not break hashing if you change the way uid is generated // to not break hashing if you change the way uid is generated
val uid = ThreadLocalRandom.current.nextInt() val uid = ThreadLocalRandom.current.nextInt()
if (uid == undefinedUid) newUid if (uid == undefinedUid) newUid()
else uid else uid
} }

View file

@ -316,8 +316,6 @@ final class ChildActorPath private[akka] (val parent: ActorPath, val name: Strin
throw new IllegalArgumentException( throw new IllegalArgumentException(
"# is a fragment separator and is not legal in ActorPath names: [%s]".format(name)) "# is a fragment separator and is not legal in ActorPath names: [%s]".format(name))
def this(parent: ActorPath, name: String) = this(parent, name, ActorCell.undefinedUid)
override def address: Address = root.address override def address: Address = root.address
override def /(child: String): ActorPath = { override def /(child: String): ActorPath = {

View file

@ -156,8 +156,8 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
} }
override def toString: String = override def toString: String =
if (path.uid == ActorCell.undefinedUid) s"Actor[${path}]" if (path.uid == ActorCell.undefinedUid) s"Actor[$path]"
else s"Actor[${path}#${path.uid}]" else s"Actor[$path#${path.uid}]"
} }
/** /**

View file

@ -9,7 +9,6 @@ import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.routing._ import akka.routing._
import akka.event._ import akka.event._
import akka.util.Helpers import akka.util.Helpers
import akka.japi.Util.immutableSeq
import akka.util.Collections.EmptyImmutableSeq import akka.util.Collections.EmptyImmutableSeq
import scala.util.control.NonFatal import scala.util.control.NonFatal
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
@ -121,41 +120,6 @@ import akka.util.OptionVal
lookupDeploy: Boolean, lookupDeploy: Boolean,
async: Boolean): InternalActorRef async: Boolean): InternalActorRef
/**
* INTERNAL API
*
* 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 references acquired with `actorFor` do not always include the full information
* about the underlying actor identity and therefore such references do not always compare
* equal to references acquired with `actorOf`, `sender`, or `context.self`.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(path: ActorPath): InternalActorRef
/**
* INTERNAL API
*
* Create actor reference for a specified local or remote path, which will
* be parsed using java.net.URI. If no such actor exists, it will be
* (equivalent to) a dead letter reference. If `s` is a relative URI, resolve
* it relative to the given ref.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(ref: InternalActorRef, s: String): InternalActorRef
/**
* INTERNAL API
*
* Create actor reference for the specified child path starting at the
* given starting point. This method always returns an actor which is logically local,
* i.e. it cannot be used to obtain a reference to an actor which is not
* physically or logically attached to this actor system.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(ref: InternalActorRef, p: Iterable[String]): InternalActorRef
/** /**
* Create actor reference for a specified path. If no such * Create actor reference for a specified 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.
@ -256,89 +220,6 @@ trait ActorRefFactory {
*/ */
def actorOf(props: Props, name: String): ActorRef def actorOf(props: Props, name: String): ActorRef
/**
* INTERNAL API
*
* Look-up an actor by path; if it does not exist, returns a reference to
* the dead-letter mailbox of the [[akka.actor.ActorSystem]]. If the path
* point to an actor which is not local, no attempt is made during this
* call to verify that the actor it represents does exist or is alive; use
* `watch(ref)` to be notified of the targets termination, which is also
* signaled if the queried path cannot be resolved.
*
* Actor references acquired with `actorFor` do not always include the full information
* about the underlying actor identity and therefore such references do not always compare
* equal to references acquired with `actorOf`, `sender`, or `context.self`.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(path: ActorPath): ActorRef = provider.actorFor(path)
/**
* INTERNAL API
*
* Look-up an actor by path represented as string.
*
* Absolute URIs like `akka://appname/user/actorA` are looked up as described
* for look-ups by `actorOf(ActorPath)`.
*
* Relative URIs like `/service/actorA/childB` are looked up relative to the
* root path of the [[akka.actor.ActorSystem]] containing this factory and as
* described for look-ups by `actorOf(Iterable[String])`.
*
* Relative URIs like `myChild/grandChild` or `../myBrother` are looked up
* relative to the current context as described for look-ups by
* `actorOf(Iterable[String])`
*
* Actor references acquired with `actorFor` do not always include the full information
* about the underlying actor identity and therefore such references do not always compare
* equal to references acquired with `actorOf`, `sender`, or `context.self`.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(path: String): ActorRef = provider.actorFor(lookupRoot, path)
/**
* INTERNAL API
*
* Look-up an actor by applying the given path elements, starting from the
* current context, where `".."` signifies the parent of an actor.
*
* Example:
* {{{
* class MyActor extends Actor {
* def receive = {
* case msg =>
* ...
* val target = context.actorFor(Seq("..", "myBrother", "myNephew"))
* ...
* }
* }
* }}}
*
* For maximum performance use a collection with efficient head & tail operations.
*
* Actor references acquired with `actorFor` do not always include the full information
* about the underlying actor identity and therefore such references do not always compare
* equal to references acquired with `actorOf`, `sender`, or `context.self`.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(path: Iterable[String]): ActorRef = provider.actorFor(lookupRoot, path)
/**
* INTERNAL API
*
* Java API: Look-up an actor by applying the given path elements, starting from the
* current context, where `".."` signifies the parent of an actor.
*
* For maximum performance use a collection with efficient head & tail operations.
*
* Actor references acquired with `actorFor` do not always include the full information
* about the underlying actor identity and therefore such references do not always compare
* equal to references acquired with `actorOf`, `sender`, or `context.self`.
*/
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] def actorFor(path: java.lang.Iterable[String]): ActorRef =
provider.actorFor(lookupRoot, immutableSeq(path))
/** /**
* Construct an [[akka.actor.ActorSelection]] from the given path, which is * Construct an [[akka.actor.ActorSelection]] from the given path, which is
* parsed for wildcards (these are replaced by regular expressions * parsed for wildcards (these are replaced by regular expressions
@ -678,41 +559,6 @@ private[akka] class LocalActorRefProvider private[akka] (
eventStream.startDefaultLoggers(_system) eventStream.startDefaultLoggers(_system)
} }
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] override def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
case RelativeActorPath(elems) =>
if (elems.isEmpty) {
log.debug("look-up of empty path string [{}] fails (per definition)", path)
deadLetters
} else if (elems.head.isEmpty) actorFor(rootGuardian, elems.tail)
else actorFor(ref, elems)
case ActorPathExtractor(address, elems) if address == rootPath.address => actorFor(rootGuardian, elems)
case _ =>
log.debug("look-up of unknown path [{}] failed", path)
deadLetters
}
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] override def actorFor(path: ActorPath): InternalActorRef =
if (path.root == rootPath) actorFor(rootGuardian, path.elements)
else {
log.debug("look-up of foreign ActorPath [{}] failed", path)
deadLetters
}
@deprecated("use actorSelection instead of actorFor", "2.2")
private[akka] override def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef =
if (path.isEmpty) {
log.debug("look-up of empty path sequence fails (per definition)")
deadLetters
} else
ref.getChild(path.iterator) match {
case Nobody =>
log.debug("look-up of path sequence [/{}] failed", path.mkString("/"))
new EmptyLocalActorRef(system.provider, ref.path / path, eventStream)
case x => x
}
def resolveActorRef(path: String): ActorRef = path match { def resolveActorRef(path: String): ActorRef = path match {
case ActorPathExtractor(address, elems) if address == rootPath.address => resolveActorRef(rootGuardian, elems) case ActorPathExtractor(address, elems) if address == rootPath.address => resolveActorRef(rootGuardian, elems)
case _ => case _ =>

View file

@ -6,10 +6,9 @@ package akka.actor.dungeon
import akka.dispatch.sysmsg.{ DeathWatchNotification, Unwatch, Watch } import akka.dispatch.sysmsg.{ DeathWatchNotification, Unwatch, Watch }
import akka.event.Logging.{ Debug, Warning } import akka.event.Logging.{ Debug, Warning }
import akka.actor.{ Actor, ActorCell, ActorRef, ActorRefScope, Address, InternalActorRef, MinimalActorRef, Terminated } import akka.actor.{ Actor, ActorCell, ActorRef, ActorRefScope, Address, InternalActorRef, Terminated }
import akka.event.AddressTerminatedTopic import akka.event.AddressTerminatedTopic
import akka.util.unused import akka.util.unused
import com.github.ghik.silencer.silent
private[akka] trait DeathWatch { this: ActorCell => private[akka] trait DeathWatch { this: ActorCell =>
@ -26,7 +25,7 @@ private[akka] trait DeathWatch { this: ActorCell =>
override final def watch(subject: ActorRef): ActorRef = subject match { override final def watch(subject: ActorRef): ActorRef = subject match {
case a: InternalActorRef => case a: InternalActorRef =>
if (a != self) { if (a != self) {
if (!watchingContains(a)) if (!watching.contains(a))
maintainAddressTerminatedSubscription(a) { maintainAddressTerminatedSubscription(a) {
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
updateWatching(a, None) updateWatching(a, None)
@ -39,7 +38,7 @@ private[akka] trait DeathWatch { this: ActorCell =>
override final def watchWith(subject: ActorRef, msg: Any): ActorRef = subject match { override final def watchWith(subject: ActorRef, msg: Any): ActorRef = subject match {
case a: InternalActorRef => case a: InternalActorRef =>
if (a != self) { if (a != self) {
if (!watchingContains(a)) if (!watching.contains(a))
maintainAddressTerminatedSubscription(a) { maintainAddressTerminatedSubscription(a) {
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
updateWatching(a, Some(msg)) updateWatching(a, Some(msg))
@ -51,13 +50,13 @@ private[akka] trait DeathWatch { this: ActorCell =>
override final def unwatch(subject: ActorRef): ActorRef = subject match { override final def unwatch(subject: ActorRef): ActorRef = subject match {
case a: InternalActorRef => case a: InternalActorRef =>
if (a != self && watchingContains(a)) { if (a != self && watching.contains(a)) {
a.sendSystemMessage(Unwatch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ a.sendSystemMessage(Unwatch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
maintainAddressTerminatedSubscription(a) { maintainAddressTerminatedSubscription(a) {
watching = removeFromMap(a, watching) watching -= a
} }
} }
terminatedQueued = removeFromMap(a, terminatedQueued) terminatedQueued -= a
a a
} }
@ -75,11 +74,11 @@ private[akka] trait DeathWatch { this: ActorCell =>
actor: ActorRef, actor: ActorRef,
existenceConfirmed: Boolean, existenceConfirmed: Boolean,
addressTerminated: Boolean): Unit = { addressTerminated: Boolean): Unit = {
watchingGet(actor) match { watching.get(actor) match {
case None => // We're apparently no longer watching this actor. case None => // We're apparently no longer watching this actor.
case Some(optionalMessage) => case Some(optionalMessage) =>
maintainAddressTerminatedSubscription(actor) { maintainAddressTerminatedSubscription(actor) {
watching = removeFromMap(actor, watching) watching -= actor
} }
if (!isTerminating) { if (!isTerminating) {
self.tell(Terminated(actor)(existenceConfirmed, addressTerminated), actor) self.tell(Terminated(actor)(existenceConfirmed, addressTerminated), actor)
@ -93,31 +92,6 @@ private[akka] trait DeathWatch { this: ActorCell =>
if (!terminatedQueued.contains(subject)) if (!terminatedQueued.contains(subject))
terminatedQueued += subject -> customMessage terminatedQueued += subject -> customMessage
// TODO this should be removed and be replaced with `watching.contains(subject)`
// when all actor references have uid, i.e. actorFor is removed
private def watchingContains(subject: ActorRef): Boolean =
watching.contains(subject) || (subject.path.uid != ActorCell.undefinedUid &&
watching.contains(new UndefinedUidActorRef(subject)))
// TODO this should be removed and be replaced with `watching.get(subject)`
// when all actor references have uid, i.e. actorFor is removed
// Returns None when the subject is not being watched.
// If the subject is being matched, the inner option is the optional custom termination
// message that should be sent instead of the default Terminated.
private def watchingGet(subject: ActorRef): Option[Option[Any]] =
watching
.get(subject)
.orElse(
if (subject.path.uid == ActorCell.undefinedUid) None
else watching.get(new UndefinedUidActorRef(subject)))
// TODO this should be removed and be replaced with `set - subject`
// when all actor references have uid, i.e. actorFor is removed
@silent
private def removeFromMap[T](subject: ActorRef, map: Map[ActorRef, T]): Map[ActorRef, T] =
if (subject.path.uid != ActorCell.undefinedUid) (map - subject) - new UndefinedUidActorRef(subject)
else map.filterKeys(_.path != subject.path).toMap
private def updateWatching(ref: InternalActorRef, newMessage: Option[Any]): Unit = private def updateWatching(ref: InternalActorRef, newMessage: Option[Any]): Unit =
watching = watching.updated(ref, newMessage) watching = watching.updated(ref, newMessage)
@ -263,8 +237,3 @@ private[akka] trait DeathWatch { this: ActorCell =>
private def subscribeAddressTerminated(): Unit = AddressTerminatedTopic(system).subscribe(self) private def subscribeAddressTerminated(): Unit = AddressTerminatedTopic(system).subscribe(self)
} }
private[akka] class UndefinedUidActorRef(ref: ActorRef) extends MinimalActorRef {
override val path = ref.path.withUid(ActorCell.undefinedUid)
override def provider = throw new UnsupportedOperationException("UndefinedUidActorRef does not provide")
}

View file

@ -25,14 +25,18 @@ and include them in your own project or library under your own package name.
If you are still using Scala 2.11 then you must upgrade to 2.12 or 2.13 If you are still using Scala 2.11 then you must upgrade to 2.12 or 2.13
### Actor DSL removal ## Actor DSL removal
Actor DSL is a rarely used feature and has been deprecated since `2.5.0`. Actor DSL is a rarely used feature and has been deprecated since `2.5.0`.
Use plain `system.actorOf` instead of the DSL to create Actors if you have been using it. Use plain `system.actorOf` instead of the DSL to create Actors if you have been using it.
## actorFor removal
`actorFor` has been deprecated since `2.2`. Use `ActorSelection` instead.
## Default remoting is now Artery TCP ## Default remoting is now Artery TCP
@ref[Artery TCP](../remoting-artery.md) is now the default remoting implementation. @ref[Artery TCP](../remoting-artery.md) is now the default remoting implementation.
Classic remoting has been deprecated and will be removed in `2.7.0`. Classic remoting has been deprecated and will be removed in `2.7.0`.
<a id="classic-to-artery"></a> <a id="classic-to-artery"></a>
@ -70,7 +74,7 @@ for how to do this.
The following defaults have changed: The following defaults have changed:
* `akka.remote.artery.transport` default has changed from `aeron-udp` to `tcp` * `akka.remote.artery.transport` default has changed from `aeron-udp` to `tcp`
The following properties have moved. If you don't adjust these from their defaults no changes are required: The following properties have moved. If you don't adjust these from their defaults no changes are required:
@ -93,13 +97,13 @@ For TCP:
### Remaining with Classic remoting (not recommended) ### Remaining with Classic remoting (not recommended)
Classic remoting is deprecated but can be used in `2.6.` Any configuration under `akka.remote` that is Classic remoting is deprecated but can be used in `2.6.` Any configuration under `akka.remote` that is
specific to classic remoting needs to be moved to `akka.remote.classic`. To see which configuration options specific to classic remoting needs to be moved to `akka.remote.classic`. To see which configuration options
are specific to classic search for them in: [`akka-remote/reference.conf`](/akka-remote/src/main/resources/reference.conf) are specific to classic search for them in: [`akka-remote/reference.conf`](/akka-remote/src/main/resources/reference.conf)
## Netty UDP has been removed ## Netty UDP has been removed
Classic remoting over UDP has been deprecated since `2.5.0` and now has been removed. Classic remoting over UDP has been deprecated since `2.5.0` and now has been removed.
To continue to use UDP configure @ref[Artery UDP](../remoting-artery.md#configuring-ssl-tls-for-akka-remoting) or migrate to Artery TCP. To continue to use UDP configure @ref[Artery UDP](../remoting-artery.md#configuring-ssl-tls-for-akka-remoting) or migrate to Artery TCP.
A full cluster restart is required to change to Artery. A full cluster restart is required to change to Artery.

View file

@ -1,3 +1,6 @@
# #26190 remove actorFor
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteActorRefProvider.actorFor")
# Make artery default and remove netty udp #26179 # Make artery default and remove netty udp #26179
ProblemFilters.exclude[MissingClassProblem]("akka.remote.transport.netty.UdpClientHandler") ProblemFilters.exclude[MissingClassProblem]("akka.remote.transport.netty.UdpClientHandler")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.transport.netty.NettyTransportSettings.TransportMode") ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.transport.netty.NettyTransportSettings.TransportMode")

View file

@ -375,52 +375,6 @@ private[akka] class RemoteActorRefProvider(
} }
} }
@deprecated("use actorSelection instead of actorFor", "2.2")
override private[akka] def actorFor(path: ActorPath): InternalActorRef = {
if (hasAddress(path.address)) actorFor(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 looking up address [{}]", path.address)
new EmptyLocalActorRef(this, path, eventStream)
}
}
@deprecated("use actorSelection instead of actorFor", "2.2")
override private[akka] def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
case ActorPathExtractor(address, elems) =>
if (hasAddress(address)) actorFor(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 looking up address [{}]", rootPath.address)
new EmptyLocalActorRef(this, rootPath, eventStream)
}
}
case _ => local.actorFor(ref, path)
}
@deprecated("use actorSelection instead of actorFor", "2.2")
override private[akka] def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef =
local.actorFor(ref, path)
def rootGuardianAt(address: Address): ActorRef = { def rootGuardianAt(address: Address): ActorRef = {
if (hasAddress(address)) rootGuardian if (hasAddress(address)) rootGuardian
else else
@ -641,17 +595,11 @@ private[akka] class RemoteActorRef private[akka] (
/** /**
* Determine if a watch/unwatch message must be handled by the remoteWatcher actor, or sent to this remote ref * Determine if a watch/unwatch message must be handled by the remoteWatcher actor, or sent to this remote ref
*/ */
def isWatchIntercepted(watchee: ActorRef, watcher: ActorRef) = def isWatchIntercepted(watchee: ActorRef, watcher: ActorRef) = {
if (watchee.path.uid == akka.actor.ActorCell.undefinedUid) { // If watchee != this then watcher should == this. This is a reverse watch, and it is not intercepted
provider.log.debug( // If watchee == this, only the watches from remoteWatcher are sent on the wire, on behalf of other watchers
"actorFor is deprecated, and watching a remote ActorRef acquired with actorFor is not reliable: [{}]", watcher != provider.remoteWatcher && watchee == this
watchee.path) }
false // Not managed by the remote watcher, so not reliable to communication failure or remote system crash
} else {
// If watchee != this then watcher should == this. This is a reverse watch, and it is not intercepted
// If watchee == this, only the watches from remoteWatcher are sent on the wire, on behalf of other watchers
watcher != provider.remoteWatcher && watchee == this
}
def sendSystemMessage(message: SystemMessage): Unit = def sendSystemMessage(message: SystemMessage): Unit =
try { try {

View file

@ -1,106 +0,0 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.remote.artery
import akka.actor.{ Actor, ActorRef, ActorRefScope, PoisonPill, Props }
import akka.pattern.ask
import akka.remote.RemoteActorRef
import akka.remote.classic.RemotingSpec.ActorForReq
import akka.testkit.{ EventFilter, _ }
import akka.util.Timeout
import com.github.ghik.silencer.silent
import scala.concurrent.duration._
object RemoteActorForSpec {
final case class ActorForReq(s: String) extends JavaSerializable
}
@silent
class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with ImplicitSender with DefaultTimeout {
val remoteSystem = newRemoteSystem()
val remotePort = port(remoteSystem)
"Remote lookups" should {
"support remote look-ups" in {
remoteSystem.actorOf(TestActors.echoActorProps, "remote-look-ups")
val remoteRef = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/remote-look-ups")
remoteRef ! "ping"
expectMsg("ping")
}
// FIXME does not log anything currently
"send warning message for wrong address" ignore {
filterEvents(EventFilter.warning(pattern = "Address is now gated for ", occurrences = 1)) {
localSystem.actorFor("akka://nonexistingsystem@localhost:12346/user/echo") ! "ping"
}
}
"support ask" in {
remoteSystem.actorOf(TestActors.echoActorProps, "support-ask")
val remoteRef = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/support-ask")
implicit val timeout: Timeout = 10.seconds
(remoteRef ? "ping").futureValue should ===("ping")
}
"send dead letters on remote if actor does not exist" in {
EventFilter
.warning(pattern = "dead.*buh", occurrences = 1)
.intercept {
localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/dead-letters-on-remote") ! "buh"
}(remoteSystem)
}
// FIXME needs remote deployment section
"look-up actors across node boundaries" ignore {
val l = localSystem.actorOf(Props(new Actor {
def receive = {
case (p: Props, n: String) => sender() ! context.actorOf(p, n)
case ActorForReq(s) => sender() ! context.actorFor(s)
}
}), "looker1")
// child is configured to be deployed on remote-sys (remoteSystem)
l ! ((TestActors.echoActorProps, "child"))
val child = expectMsgType[ActorRef]
// grandchild is configured to be deployed on RemotingSpec (system)
child ! ((TestActors.echoActorProps, "grandchild"))
val grandchild = expectMsgType[ActorRef]
grandchild.asInstanceOf[ActorRefScope].isLocal should ===(true)
grandchild ! 43
expectMsg(43)
val myref = localSystem.actorFor(system / "looker1" / "child" / "grandchild")
myref.isInstanceOf[RemoteActorRef] should ===(true)
myref ! 44
expectMsg(44)
lastSender should ===(grandchild)
(lastSender should be).theSameInstanceAs(grandchild)
child.asInstanceOf[RemoteActorRef].getParent should ===(l)
(localSystem.actorFor("/user/looker1/child") should be).theSameInstanceAs(child)
(l ? ActorForReq("child/..")).mapTo[AnyRef].futureValue should be.theSameInstanceAs(l)
(localSystem.actorFor(system / "looker1" / "child") ? ActorForReq("..")).mapTo[AnyRef].futureValue should be
.theSameInstanceAs(l)
watch(child)
child ! PoisonPill
expectMsg("postStop")
expectTerminated(child)
l ! ((TestActors.echoActorProps, "child"))
val child2 = expectMsgType[ActorRef]
child2 ! 45
expectMsg(45)
// msg to old ActorRef (different uid) should not get through
child2.path.uid should not be (child.path.uid)
child ! 46
expectNoMessage(1.second)
system.actorFor(system / "looker1" / "child") ! 47
expectMsg(47)
}
}
}

View file

@ -80,10 +80,12 @@ class RemoteDeathWatchSpec
"receive Terminated when watched node is unknown host" in { "receive Terminated when watched node is unknown host" in {
val path = RootActorPath(Address("akka", system.name, "unknownhost", 2552)) / "user" / "subject" val path = RootActorPath(Address("akka", system.name, "unknownhost", 2552)) / "user" / "subject"
system.actorOf(Props(new Actor { system.actorOf(Props(new Actor {
@silent @silent
val watchee = context.actorFor(path) val watchee = RARP(context.system).provider.resolveActorRef(path)
context.watch(watchee) context.watch(watchee)
def receive = { def receive = {
case t: Terminated => testActor ! t.actor.path case t: Terminated => testActor ! t.actor.path
} }

View file

@ -95,7 +95,8 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec("""
} }
}), bigBounceId) }), bigBounceId)
@silent @silent
val bigBounceHere = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/$bigBounceId") val bigBounceHere =
RARP(system).provider.resolveActorRef(s"akka://${remoteSystem.name}@localhost:$remotePort/user/$bigBounceId")
val eventForwarder = localSystem.actorOf(Props(new Actor { val eventForwarder = localSystem.actorOf(Props(new Actor {
def receive = { def receive = {

View file

@ -85,10 +85,12 @@ akka {
"receive Terminated when watched node is unknown host" in { "receive Terminated when watched node is unknown host" in {
val path = RootActorPath(Address(protocol, system.name, "unknownhost", 2552)) / "user" / "subject" val path = RootActorPath(Address(protocol, system.name, "unknownhost", 2552)) / "user" / "subject"
system.actorOf(Props(new Actor { system.actorOf(Props(new Actor {
@silent @silent
val watchee = context.actorFor(path) val watchee = RARP(context.system).provider.resolveActorRef(path)
context.watch(watchee) context.watch(watchee)
def receive = { def receive = {
case t: Terminated => testActor ! t.actor.path case t: Terminated => testActor ! t.actor.path
} }

View file

@ -25,7 +25,6 @@ import scala.concurrent.duration._
object RemotingSpec { object RemotingSpec {
final case class ActorForReq(s: String)
final case class ActorSelReq(s: String) final case class ActorSelReq(s: String)
class Echo1 extends Actor { class Echo1 extends Actor {
@ -34,13 +33,8 @@ object RemotingSpec {
def receive = { def receive = {
case (_: Props, n: String) => sender() ! context.actorOf(Props[Echo1], n) case (_: Props, n: String) => sender() ! context.actorOf(Props[Echo1], n)
case ex: Exception => throw ex case ex: Exception => throw ex
case ActorForReq(s) => { case ActorSelReq(s) => sender() ! context.actorSelection(s)
@silent case x => target = sender(); sender() ! x
val actor = context.actorFor(s)
sender() ! actor
}
case ActorSelReq(s) => sender() ! context.actorSelection(s)
case x => target = sender(); sender() ! x
} }
override def preStart(): Unit = {} override def preStart(): Unit = {}
@ -161,7 +155,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
val remote = remoteSystem.actorOf(Props[Echo2], "echo") val remote = remoteSystem.actorOf(Props[Echo2], "echo")
val here = system.actorFor("akka.test://remote-sys@localhost:12346/user/echo") val here = RARP(system).provider.resolveActorRef("akka.test://remote-sys@localhost:12346/user/echo")
private def verifySend(msg: Any)(afterSend: => Unit): Unit = { private def verifySend(msg: Any)(afterSend: => Unit): Unit = {
val bigBounceId = s"bigBounce-${ThreadLocalRandom.current.nextInt()}" val bigBounceId = s"bigBounce-${ThreadLocalRandom.current.nextInt()}"
@ -171,7 +165,8 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
case x => sender() ! x case x => sender() ! x
} }
}).withDeploy(Deploy.local), bigBounceId) }).withDeploy(Deploy.local), bigBounceId)
val bigBounceHere = system.actorFor(s"akka.test://remote-sys@localhost:12346/user/$bigBounceId") val bigBounceHere =
RARP(system).provider.resolveActorRef(s"akka.test://remote-sys@localhost:12346/user/$bigBounceId")
val eventForwarder = system.actorOf(Props(new Actor { val eventForwarder = system.actorOf(Props(new Actor {
def receive = { def receive = {
@ -219,7 +214,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
"send warning message for wrong address" in { "send warning message for wrong address" in {
filterEvents(EventFilter.warning(pattern = "Address is now gated for ", occurrences = 1)) { filterEvents(EventFilter.warning(pattern = "Address is now gated for ", occurrences = 1)) {
system.actorFor("akka.test://nonexistingsystem@localhost:12346/user/echo") ! "ping" RARP(system).provider.resolveActorRef("akka.test://nonexistingsystem@localhost:12346/user/echo") ! "ping"
} }
} }
@ -234,7 +229,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
EventFilter EventFilter
.warning(pattern = "dead.*buh", occurrences = 1) .warning(pattern = "dead.*buh", occurrences = 1)
.intercept { .intercept {
system.actorFor("akka.test://remote-sys@localhost:12346/does/not/exist") ! "buh" RARP(system).provider.resolveActorRef("akka.test://remote-sys@localhost:12346/does/not/exist") ! "buh"
}(remoteSystem) }(remoteSystem)
} }
@ -316,10 +311,10 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
echo ! 74 echo ! 74
expectNoMessage(1.second) expectNoMessage(1.second)
remoteSystem.actorFor("/user/otherEcho1") ! 75 remoteSystem.actorSelection("/user/otherEcho1") ! 75
expectMsg(75) expectMsg(75)
system.actorFor("akka.test://remote-sys@localhost:12346/user/otherEcho1") ! 76 system.actorSelection("akka.test://remote-sys@localhost:12346/user/otherEcho1") ! 76
expectMsg(76) expectMsg(76)
remoteSystem.actorSelection("/user/otherEcho1") ! 77 remoteSystem.actorSelection("/user/otherEcho1") ! 77
@ -329,53 +324,6 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsg(78) expectMsg(78)
} }
"look-up actors across node boundaries" in {
val l = system.actorOf(Props(new Actor {
def receive = {
case (p: Props, n: String) => sender() ! context.actorOf(p, n)
case ActorForReq(s) => {
sender() ! context.actorFor(s)
}
}
}), "looker1")
// child is configured to be deployed on remote-sys (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 should ===(true)
grandchild ! 43
expectMsg(43)
val myref = system.actorFor(system / "looker1" / "child" / "grandchild")
myref.isInstanceOf[RemoteActorRef] should ===(true)
myref ! 44
expectMsg(44)
lastSender should ===(grandchild)
(lastSender should be).theSameInstanceAs(grandchild)
child.asInstanceOf[RemoteActorRef].getParent should ===(l)
(system.actorFor("/user/looker1/child") should be).theSameInstanceAs(child)
(Await.result(l ? ActorForReq("child/.."), timeout.duration).asInstanceOf[AnyRef] should be).theSameInstanceAs(l)
(Await
.result(system.actorFor(system / "looker1" / "child") ? ActorForReq(".."), timeout.duration)
.asInstanceOf[AnyRef] should be).theSameInstanceAs(l)
watch(child)
child ! PoisonPill
expectMsg("postStop")
expectTerminated(child)
l ! ((Props[Echo1], "child"))
val child2 = expectMsgType[ActorRef]
child2 ! 45
expectMsg(45)
// msg to old ActorRef (different uid) should not get through
child2.path.uid should not be (child.path.uid)
child ! 46
expectNoMessage(1.second)
system.actorFor(system / "looker1" / "child") ! 47
expectMsg(47)
}
"select actors across node boundaries" in { "select actors across node boundaries" in {
val l = system.actorOf(Props(new Actor { val l = system.actorOf(Props(new Actor {
def receive = { def receive = {
@ -562,14 +510,16 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
// check that we use the specified transport address instead of the default // check that we use the specified transport address instead of the default
val otherGuyRemoteTcp = otherGuy.path.toSerializationFormatWithAddress(getOtherAddress(otherSystem, "tcp")) val otherGuyRemoteTcp = otherGuy.path.toSerializationFormatWithAddress(getOtherAddress(otherSystem, "tcp"))
val remoteEchoHereTcp = val remoteEchoHereTcp =
system.actorFor(s"akka.tcp://remote-sys@localhost:${port(remoteSystem, "tcp")}/user/echo") RARP(system).provider
.resolveActorRef(s"akka.tcp://remote-sys@localhost:${port(remoteSystem, "tcp")}/user/echo")
val proxyTcp = system.actorOf(Props(classOf[Proxy], remoteEchoHereTcp, testActor), "proxy-tcp") val proxyTcp = system.actorOf(Props(classOf[Proxy], remoteEchoHereTcp, testActor), "proxy-tcp")
proxyTcp ! otherGuy proxyTcp ! otherGuy
expectMsg(3.seconds, ("pong", otherGuyRemoteTcp)) expectMsg(3.seconds, ("pong", otherGuyRemoteTcp))
// now check that we fall back to default when we haven't got a corresponding transport // now check that we fall back to default when we haven't got a corresponding transport
val otherGuyRemoteTest = otherGuy.path.toSerializationFormatWithAddress(getOtherAddress(otherSystem, "test")) val otherGuyRemoteTest = otherGuy.path.toSerializationFormatWithAddress(getOtherAddress(otherSystem, "test"))
val remoteEchoHereSsl = val remoteEchoHereSsl =
system.actorFor(s"akka.ssl.tcp://remote-sys@localhost:${port(remoteSystem, "ssl.tcp")}/user/echo") RARP(system).provider
.resolveActorRef(s"akka.ssl.tcp://remote-sys@localhost:${port(remoteSystem, "ssl.tcp")}/user/echo")
val proxySsl = system.actorOf(Props(classOf[Proxy], remoteEchoHereSsl, testActor), "proxy-ssl") val proxySsl = system.actorOf(Props(classOf[Proxy], remoteEchoHereSsl, testActor), "proxy-ssl")
EventFilter EventFilter
.warning(start = "Error while resolving ActorRef", occurrences = 1) .warning(start = "Error while resolving ActorRef", occurrences = 1)