Skip to content

Commit

Permalink
Merge pull request akka#19487 from akka/wip-19014-make-StageActorRef-…
Browse files Browse the repository at this point in the history
…location-transparent-RK

Wip 19014 make stage actor ref location transparent rk
  • Loading branch information
rkuhn committed Jan 19, 2016
2 parents 1caf4b2 + f7444d6 commit cc2daa5
Show file tree
Hide file tree
Showing 15 changed files with 423 additions and 166 deletions.
100 changes: 100 additions & 0 deletions akka-actor-tests/src/test/scala/akka/actor/FunctionRefSpec.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
/**
* Copyright (C) 2016 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor

import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import scala.concurrent.duration._
import akka.testkit.EventFilter

object FunctionRefSpec {

case class GetForwarder(replyTo: ActorRef)
case class DropForwarder(ref: FunctionRef)
case class Forwarded(msg: Any, sender: ActorRef)

class Super extends Actor {
def receive = {
case GetForwarder(replyTo)
val cell = context.asInstanceOf[ActorCell]
val ref = cell.addFunctionRef((sender, msg) replyTo ! Forwarded(msg, sender))
replyTo ! ref
case DropForwarder(ref)
val cell = context.asInstanceOf[ActorCell]
cell.removeFunctionRef(ref)
}
}

class SupSuper extends Actor {
val s = context.actorOf(Props[Super], "super")
def receive = {
case msg s ! msg
}
}

}

@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class FunctionRefSpec extends AkkaSpec with ImplicitSender {
import FunctionRefSpec._

def commonTests(s: ActorRef) = {
s ! GetForwarder(testActor)
val forwarder = expectMsgType[FunctionRef]

"forward messages" in {
forwarder ! "hello"
expectMsg(Forwarded("hello", testActor))
}

"be watchable" in {
s ! GetForwarder(testActor)
val f = expectMsgType[FunctionRef]
watch(f)
s ! DropForwarder(f)
expectTerminated(f)
}

"be able to watch" in {
s ! GetForwarder(testActor)
val f = expectMsgType[FunctionRef]
forwarder.watch(f)
s ! DropForwarder(f)
expectMsg(Forwarded(Terminated(f)(true, false), null))
}

"terminate when their parent terminates" in {
watch(forwarder)
s ! PoisonPill
expectTerminated(forwarder)
}
}

"A FunctionRef" when {

"created by a toplevel actor" must {
val s = system.actorOf(Props[Super], "super")
commonTests(s)
}

"created by a non-toplevel actor" must {
val s = system.actorOf(Props[SupSuper], "supsuper")
commonTests(s)
}

"not registered" must {
"not be found" in {
val provider = system.asInstanceOf[ExtendedActorSystem].provider
val ref = new FunctionRef(testActor.path / "blabla", provider, system.eventStream, (x, y) ())
EventFilter[ClassCastException](occurrences = 1) intercept {
// needs to be something that fails when the deserialized form is not a FunctionRef
// this relies upon serialize-messages during tests
testActor ! DropForwarder(ref)
expectNoMsg(1.second)
}
}
}

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -11,12 +11,14 @@ final class AbstractActorCell {
final static long mailboxOffset;
final static long childrenOffset;
final static long nextNameOffset;
final static long functionRefsOffset;

static {
try {
mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Dispatch$$_mailboxDoNotCallMeDirectly"));
childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Children$$_childrenRefsDoNotCallMeDirectly"));
nextNameOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Children$$_nextNameDoNotCallMeDirectly"));
functionRefsOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Children$$_functionRefsDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}
Expand Down
140 changes: 139 additions & 1 deletion akka-actor/src/main/scala/akka/actor/ActorRef.scala
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,9 @@ import akka.serialization.{ Serialization, JavaSerializer }
import akka.event.EventStream
import scala.annotation.tailrec
import java.util.concurrent.ConcurrentHashMap
import akka.event.LoggingAdapter
import akka.event.{ Logging, LoggingAdapter }
import java.util.concurrent.atomic.AtomicReference
import scala.util.control.NonFatal

object ActorRef {

Expand Down Expand Up @@ -687,3 +689,139 @@ private[akka] class VirtualPathContainer(
while (iter.hasNext) f(iter.next)
}
}

/**
* INTERNAL API
*
* This kind of ActorRef passes all received messages to the given function for
* performing a non-blocking side-effect. The intended use is to transform the
* message before sending to the real target actor. Such references can be created
* by calling `ActorCell.addFunctionRef` and must be deregistered when no longer
* needed by calling `ActorCell.removeFunctionRef`. FunctionRefs do not count
* towards the live children of an actor, they do not receive the Terminate command
* and do not prevent the parent from terminating. FunctionRef is properly
* registered for remote lookup and ActorSelection.
*
* When using the watch() feature you must ensure that upon reception of the
* Terminated message the watched actorRef is unwatch()ed.
*/
private[akka] final class FunctionRef(override val path: ActorPath,
override val provider: ActorRefProvider,
val eventStream: EventStream,
f: (ActorRef, Any) Unit) extends MinimalActorRef {

override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = {
f(sender, message)
}

override def sendSystemMessage(message: SystemMessage): Unit = {
message match {
case w: Watch addWatcher(w.watchee, w.watcher)
case u: Unwatch remWatcher(u.watchee, u.watcher)
case DeathWatchNotification(actorRef, _, _)
this.!(Terminated(actorRef)(existenceConfirmed = true, addressTerminated = false))
case _ //ignore all other messages
}
}

private[this] var watching = ActorCell.emptyActorRefSet
private[this] val _watchedBy = new AtomicReference[Set[ActorRef]](ActorCell.emptyActorRefSet)

override def isTerminated = _watchedBy.get() == null

//noinspection EmptyCheck
protected def sendTerminated(): Unit = {
val watchedBy = _watchedBy.getAndSet(null)
if (watchedBy != null) {
if (watchedBy.nonEmpty) {
watchedBy foreach sendTerminated(ifLocal = false)
watchedBy foreach sendTerminated(ifLocal = true)
}
if (watching.nonEmpty) {
watching foreach unwatchWatched
watching = Set.empty
}
}
}

private def sendTerminated(ifLocal: Boolean)(watcher: ActorRef): Unit =
if (watcher.asInstanceOf[ActorRefScope].isLocal == ifLocal)
watcher.asInstanceOf[InternalActorRef].sendSystemMessage(DeathWatchNotification(this, existenceConfirmed = true, addressTerminated = false))

private def unwatchWatched(watched: ActorRef): Unit =
watched.asInstanceOf[InternalActorRef].sendSystemMessage(Unwatch(watched, this))

override def stop(): Unit = sendTerminated()

@tailrec private def addWatcher(watchee: ActorRef, watcher: ActorRef): Unit =
_watchedBy.get() match {
case null
sendTerminated(ifLocal = true)(watcher)
sendTerminated(ifLocal = false)(watcher)

case watchedBy
val watcheeSelf = watchee == this
val watcherSelf = watcher == this

if (watcheeSelf && !watcherSelf) {
if (!watchedBy.contains(watcher))
if (!_watchedBy.compareAndSet(watchedBy, watchedBy + watcher))
addWatcher(watchee, watcher) // try again
} else if (!watcheeSelf && watcherSelf) {
publish(Logging.Warning(path.toString, classOf[FunctionRef], s"externally triggered watch from $watcher to $watchee is illegal on FunctionRef"))
} else {
publish(Logging.Error(path.toString, classOf[FunctionRef], s"BUG: illegal Watch($watchee,$watcher) for $this"))
}
}

@tailrec private def remWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
_watchedBy.get() match {
case null // do nothing...
case watchedBy
val watcheeSelf = watchee == this
val watcherSelf = watcher == this

if (watcheeSelf && !watcherSelf) {
if (watchedBy.contains(watcher))
if (!_watchedBy.compareAndSet(watchedBy, watchedBy - watcher))
remWatcher(watchee, watcher) // try again
} else if (!watcheeSelf && watcherSelf) {
publish(Logging.Warning(path.toString, classOf[FunctionRef], s"externally triggered unwatch from $watcher to $watchee is illegal on FunctionRef"))
} else {
publish(Logging.Error(path.toString, classOf[FunctionRef], s"BUG: illegal Unwatch($watchee,$watcher) for $this"))
}
}
}

private def publish(e: Logging.LogEvent): Unit = try eventStream.publish(e) catch { case NonFatal(_) }

/**
* Have this FunctionRef watch the given Actor. This method must not be
* called concurrently from different threads, it should only be called by
* its parent Actor.
*
* Upon receiving the Terminated message, unwatch() must be called from a
* safe context (i.e. normally from the parent Actor).
*/
def watch(actorRef: ActorRef): Unit = {
watching += actorRef
actorRef.asInstanceOf[InternalActorRef].sendSystemMessage(Watch(actorRef.asInstanceOf[InternalActorRef], this))
}

/**
* Have this FunctionRef unwatch the given Actor. This method must not be
* called concurrently from different threads, it should only be called by
* its parent Actor.
*/
def unwatch(actorRef: ActorRef): Unit = {
watching -= actorRef
actorRef.asInstanceOf[InternalActorRef].sendSystemMessage(Unwatch(actorRef.asInstanceOf[InternalActorRef], this))
}

/**
* Query whether this FunctionRef is currently watching the given Actor. This
* method must not be called concurrently from different threads, it should
* only be called by its parent Actor.
*/
def isWatching(actorRef: ActorRef): Boolean = watching.contains(actorRef)
}
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,10 @@ private[akka] class RepointableActorRef(
lookup.getChildByName(childName) match {
case Some(crs: ChildRestartStats) if uid == ActorCell.undefinedUid || uid == crs.uid
crs.child.asInstanceOf[InternalActorRef].getChild(name)
case _ Nobody
case _ lookup match {
case ac: ActorCell ac.getFunctionRefOrNobody(childName, uid)
case _ Nobody
}
}
}
} else this
Expand Down
69 changes: 61 additions & 8 deletions akka-actor/src/main/scala/akka/actor/dungeon/Children.scala
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,10 @@ import akka.serialization.SerializationExtension
import akka.util.{ Unsafe, Helpers }
import akka.serialization.SerializerWithStringManifest

private[akka] object Children {
val GetNobody = () Nobody
}

private[akka] trait Children { this: ActorCell

import ChildrenContainer._
Expand Down Expand Up @@ -41,14 +45,63 @@ private[akka] trait Children { this: ActorCell ⇒
private[akka] def attachChild(props: Props, name: String, systemService: Boolean): ActorRef =
makeChild(this, props, checkName(name), async = true, systemService = systemService)

@volatile private var _functionRefsDoNotCallMeDirectly = Map.empty[String, FunctionRef]
private def functionRefs: Map[String, FunctionRef] =
Unsafe.instance.getObjectVolatile(this, AbstractActorCell.functionRefsOffset).asInstanceOf[Map[String, FunctionRef]]

private[akka] def getFunctionRefOrNobody(name: String, uid: Int = ActorCell.undefinedUid): InternalActorRef =
functionRefs.getOrElse(name, Children.GetNobody()) match {
case f: FunctionRef
if (uid == ActorCell.undefinedUid || f.path.uid == uid) f else Nobody
case other
other
}

private[akka] def addFunctionRef(f: (ActorRef, Any) Unit): FunctionRef = {
val childPath = new ChildActorPath(self.path, randomName(new java.lang.StringBuilder("$$")), ActorCell.newUid())
val ref = new FunctionRef(childPath, provider, system.eventStream, f)

@tailrec def rec(): Unit = {
val old = functionRefs
val added = old.updated(childPath.name, ref)
if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.functionRefsOffset, old, added)) rec()
}
rec()

ref
}

private[akka] def removeFunctionRef(ref: FunctionRef): Boolean = {
require(ref.path.parent eq self.path, "trying to remove FunctionRef from wrong ActorCell")
val name = ref.path.name
@tailrec def rec(): Boolean = {
val old = functionRefs
if (!old.contains(name)) false
else {
val removed = old - name
if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.functionRefsOffset, old, removed)) rec()
else {
ref.stop()
true
}
}
}
rec()
}

protected def stopFunctionRefs(): Unit = {
val refs = Unsafe.instance.getAndSetObject(this, AbstractActorCell.functionRefsOffset, Map.empty).asInstanceOf[Map[String, FunctionRef]]
refs.valuesIterator.foreach(_.stop())
}

@volatile private var _nextNameDoNotCallMeDirectly = 0L
final protected def randomName(sb: java.lang.StringBuilder): String = {
val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1)
Helpers.base64(num, sb)
}
final protected def randomName(): String = {
@tailrec def inc(): Long = {
val current = Unsafe.instance.getLongVolatile(this, AbstractActorCell.nextNameOffset)
if (Unsafe.instance.compareAndSwapLong(this, AbstractActorCell.nextNameOffset, current, current + 1)) current
else inc()
}
Helpers.base64(inc())
val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1)
Helpers.base64(num)
}

final def stop(actor: ActorRef): Unit = {
Expand Down Expand Up @@ -140,14 +193,14 @@ private[akka] trait Children { this: ActorCell ⇒
// optimization for the non-uid case
getChildByName(name) match {
case Some(crs: ChildRestartStats) crs.child.asInstanceOf[InternalActorRef]
case _ Nobody
case _ getFunctionRefOrNobody(name)
}
} else {
val (childName, uid) = ActorCell.splitNameAndUid(name)
getChildByName(childName) match {
case Some(crs: ChildRestartStats) if uid == ActorCell.undefinedUid || uid == crs.uid
crs.child.asInstanceOf[InternalActorRef]
case _ Nobody
case _ getFunctionRefOrNobody(childName, uid)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
catch handleNonFatalOrInterruptedException { e publish(Error(e, self.path.toString, clazz(a), e.getMessage)) }
finally try dispatcher.detach(this)
finally try parent.sendSystemMessage(DeathWatchNotification(self, existenceConfirmed = true, addressTerminated = false))
finally try stopFunctionRefs()
finally try tellWatchersWeDied()
finally try unwatchWatchedActors(a) // stay here as we expect an emergency stop from handleInvokeFailure
finally {
Expand Down
Loading

0 comments on commit cc2daa5

Please sign in to comment.