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
final case class FastActor(latch: TestLatch, testActor: ActorRef) extends Actor {
val ref1 = context.actorOf(Props.empty)
val ref2 = context.actorFor(ref1.path.toString)
testActor ! ref2.getClass
context.actorSelection(ref1.path.toString).tell(Identify(ref1), testActor)
latch.countDown()
def receive = {
@ -293,7 +292,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
"find actors that just have been created" in {
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 {

View file

@ -37,20 +37,16 @@ object LocalActorRefProviderSpec {
class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.config) {
"An LocalActorRefProvider" must {
"find actor refs using actorFor" 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 {
"find child actor with URL encoded name" in {
val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552"
val a = system.actorOf(Props(new Actor {
val child = context.actorOf(Props.empty, name = childName)
def receive = {
case "lookup" =>
if (childName == child.path.name) sender() ! context.actorFor(childName)
else sender() ! s"$childName is not ${child.path.name}!"
if (childName == 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)

View file

@ -26,8 +26,7 @@ class AskSpec extends AkkaSpec {
}
"return broken promises on DeadLetters" in {
val dead = system.actorFor("/system/deadLetters")
val f = dead.ask(42)(1 second)
val f = system.deadLetters.ask(42)(1 second)
f.isCompleted should ===(true)
f.value.get match {
case Failure(_: AskTimeoutException) =>
@ -37,7 +36,7 @@ class AskSpec extends AkkaSpec {
"return broken promises on EmptyLocalActorRefs" in {
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
f.isCompleted should ===(true)
f.value.get match {

View file

@ -168,9 +168,9 @@ class ConfiguredLocalRoutingSpec
"not get confused when trying to wildcard-configure children" in {
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
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)
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.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
// to not break hashing if you change the way uid is generated
val uid = ThreadLocalRandom.current.nextInt()
if (uid == undefinedUid) newUid
if (uid == undefinedUid) newUid()
else uid
}

View file

@ -316,8 +316,6 @@ final class ChildActorPath private[akka] (val parent: ActorPath, val name: Strin
throw new IllegalArgumentException(
"# 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 /(child: String): ActorPath = {

View file

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

View file

@ -9,7 +9,6 @@ import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.routing._
import akka.event._
import akka.util.Helpers
import akka.japi.Util.immutableSeq
import akka.util.Collections.EmptyImmutableSeq
import scala.util.control.NonFatal
import java.util.concurrent.atomic.AtomicLong
@ -121,41 +120,6 @@ import akka.util.OptionVal
lookupDeploy: Boolean,
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
* actor exists, it will be (equivalent to) a dead letter reference.
@ -256,89 +220,6 @@ trait ActorRefFactory {
*/
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
* parsed for wildcards (these are replaced by regular expressions
@ -678,41 +559,6 @@ private[akka] class LocalActorRefProvider private[akka] (
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 {
case ActorPathExtractor(address, elems) if address == rootPath.address => resolveActorRef(rootGuardian, elems)
case _ =>

View file

@ -6,10 +6,9 @@ package akka.actor.dungeon
import akka.dispatch.sysmsg.{ DeathWatchNotification, Unwatch, Watch }
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.util.unused
import com.github.ghik.silencer.silent
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 {
case a: InternalActorRef =>
if (a != self) {
if (!watchingContains(a))
if (!watching.contains(a))
maintainAddressTerminatedSubscription(a) {
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
updateWatching(a, None)
@ -39,7 +38,7 @@ private[akka] trait DeathWatch { this: ActorCell =>
override final def watchWith(subject: ActorRef, msg: Any): ActorRef = subject match {
case a: InternalActorRef =>
if (a != self) {
if (!watchingContains(a))
if (!watching.contains(a))
maintainAddressTerminatedSubscription(a) {
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
updateWatching(a, Some(msg))
@ -51,13 +50,13 @@ private[akka] trait DeathWatch { this: ActorCell =>
override final def unwatch(subject: ActorRef): ActorRef = subject match {
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 ⬅⬅⬅
maintainAddressTerminatedSubscription(a) {
watching = removeFromMap(a, watching)
watching -= a
}
}
terminatedQueued = removeFromMap(a, terminatedQueued)
terminatedQueued -= a
a
}
@ -75,11 +74,11 @@ private[akka] trait DeathWatch { this: ActorCell =>
actor: ActorRef,
existenceConfirmed: Boolean,
addressTerminated: Boolean): Unit = {
watchingGet(actor) match {
watching.get(actor) match {
case None => // We're apparently no longer watching this actor.
case Some(optionalMessage) =>
maintainAddressTerminatedSubscription(actor) {
watching = removeFromMap(actor, watching)
watching -= actor
}
if (!isTerminating) {
self.tell(Terminated(actor)(existenceConfirmed, addressTerminated), actor)
@ -93,31 +92,6 @@ private[akka] trait DeathWatch { this: ActorCell =>
if (!terminatedQueued.contains(subject))
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 =
watching = watching.updated(ref, newMessage)
@ -263,8 +237,3 @@ private[akka] trait DeathWatch { this: ActorCell =>
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,11 +25,15 @@ 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
### Actor DSL removal
## Actor DSL removal
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.
## actorFor removal
`actorFor` has been deprecated since `2.2`. Use `ActorSelection` instead.
## Default remoting is now Artery TCP
@ref[Artery TCP](../remoting-artery.md) is now the default remoting implementation.

View file

@ -1,3 +1,6 @@
# #26190 remove actorFor
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteActorRefProvider.actorFor")
# Make artery default and remove netty udp #26179
ProblemFilters.exclude[MissingClassProblem]("akka.remote.transport.netty.UdpClientHandler")
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 = {
if (hasAddress(address)) rootGuardian
else
@ -641,13 +595,7 @@ 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
*/
def isWatchIntercepted(watchee: ActorRef, watcher: ActorRef) =
if (watchee.path.uid == akka.actor.ActorCell.undefinedUid) {
provider.log.debug(
"actorFor is deprecated, and watching a remote ActorRef acquired with actorFor is not reliable: [{}]",
watchee.path)
false // Not managed by the remote watcher, so not reliable to communication failure or remote system crash
} else {
def isWatchIntercepted(watchee: ActorRef, watcher: ActorRef) = {
// 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

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 {
val path = RootActorPath(Address("akka", system.name, "unknownhost", 2552)) / "user" / "subject"
system.actorOf(Props(new Actor {
@silent
val watchee = context.actorFor(path)
val watchee = RARP(context.system).provider.resolveActorRef(path)
context.watch(watchee)
def receive = {
case t: Terminated => testActor ! t.actor.path
}

View file

@ -95,7 +95,8 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec("""
}
}), bigBounceId)
@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 {
def receive = {

View file

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

View file

@ -25,7 +25,6 @@ import scala.concurrent.duration._
object RemotingSpec {
final case class ActorForReq(s: String)
final case class ActorSelReq(s: String)
class Echo1 extends Actor {
@ -34,11 +33,6 @@ object RemotingSpec {
def receive = {
case (_: Props, n: String) => sender() ! context.actorOf(Props[Echo1], n)
case ex: Exception => throw ex
case ActorForReq(s) => {
@silent
val actor = context.actorFor(s)
sender() ! actor
}
case ActorSelReq(s) => sender() ! context.actorSelection(s)
case x => target = sender(); sender() ! x
}
@ -161,7 +155,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
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 = {
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
}
}).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 {
def receive = {
@ -219,7 +214,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
"send warning message for wrong address" in {
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
.warning(pattern = "dead.*buh", occurrences = 1)
.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)
}
@ -316,10 +311,10 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
echo ! 74
expectNoMessage(1.second)
remoteSystem.actorFor("/user/otherEcho1") ! 75
remoteSystem.actorSelection("/user/otherEcho1") ! 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)
remoteSystem.actorSelection("/user/otherEcho1") ! 77
@ -329,53 +324,6 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
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 {
val l = system.actorOf(Props(new Actor {
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
val otherGuyRemoteTcp = otherGuy.path.toSerializationFormatWithAddress(getOtherAddress(otherSystem, "tcp"))
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")
proxyTcp ! otherGuy
expectMsg(3.seconds, ("pong", otherGuyRemoteTcp))
// 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 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")
EventFilter
.warning(start = "Error while resolving ActorRef", occurrences = 1)