Merge pull request #511 from akka/wip-2134-deathwatch2.0-√

Wip 2134 deathwatch2.0 √
This commit is contained in:
viktorklang 2012-06-05 03:33:01 -07:00
commit 9581ec8403
28 changed files with 334 additions and 1010 deletions

View file

@ -36,7 +36,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
"notify with one Terminated message when an Actor is stopped" in {
val terminal = system.actorOf(Props.empty)
startWatching(terminal) ! "hallo"
expectMsg("hallo") // this ensures that the DaemonMsgWatch has been received before we send the PoisonPill
expectMsg("hallo")
terminal ! PoisonPill

View file

@ -374,7 +374,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ll.self.path compareTo rr.self.path }
} foreach {
case cell: ActorCell
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain()))
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain(null)))
}
System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages)

View file

@ -73,7 +73,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
watch(router)
watch(c2)
system.stop(c2)
expectMsg(Terminated(c2))
expectMsgPF() {
case t @ Terminated(`c2`) if t.existenceConfirmed == true t
}
// it might take a while until the Router has actually processed the Terminated message
awaitCond {
router ! ""
@ -84,7 +86,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
res == Seq(c1, c1)
}
system.stop(c1)
expectMsg(Terminated(router))
expectMsgPF() {
case t @ Terminated(`router`) if t.existenceConfirmed == true t
}
}
"be able to send their routees" in {

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor;
import akka.util.Unsafe;
final class AbstractActorCell {
final static long mailboxOffset;
static {
try {
mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_mailboxDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}
}
}

View file

@ -8,10 +8,12 @@ import akka.util.Unsafe;
final class AbstractPromiseActorRef {
final static long stateOffset;
final static long watchedByOffset;
static {
try {
stateOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_stateDoNotCallMeDirectly"));
watchedByOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_watchedByDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}

View file

@ -59,7 +59,7 @@ case object Kill extends Kill {
/**
* When Death Watch is used, the watcher will receive a Terminated(watched) message when watched is terminated.
*/
case class Terminated(@BeanProperty actor: ActorRef) extends PossiblyHarmful
case class Terminated(@BeanProperty actor: ActorRef)(@BeanProperty val existenceConfirmed: Boolean)
abstract class ReceiveTimeout extends PossiblyHarmful

View file

@ -6,16 +6,15 @@ package akka.actor
import akka.dispatch._
import scala.annotation.tailrec
import scala.collection.immutable.{ Stack, TreeMap }
import java.util.concurrent.TimeUnit
import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.event.Logging.{ Debug, Warning, Error }
import akka.util.{ Duration, Helpers }
import akka.japi.Procedure
import java.io.{ NotSerializableException, ObjectOutputStream }
import akka.serialization.SerializationExtension
import akka.util.NonFatal
import akka.event.Logging.LogEventException
import collection.immutable.{ TreeSet, Stack, TreeMap }
import akka.util.{ Unsafe, Duration, Helpers, NonFatal }
//TODO: everything here for current compatibility - could be limited more
@ -187,6 +186,8 @@ private[akka] object ActorCell {
final val behaviorStackPlaceHolder: Stack[Actor.Receive] = Stack.empty.push(Actor.emptyBehavior)
final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty
sealed trait SuspendReason
case object UserRequest extends SuspendReason
case class Recreation(cause: Throwable) extends SuspendReason
@ -317,7 +318,7 @@ private[akka] class ActorCell(
val props: Props,
@volatile var parent: InternalActorRef,
/*no member*/ _receiveTimeout: Option[Duration]) extends UntypedActorContext {
import AbstractActorCell.mailboxOffset
import ActorCell._
final def systemImpl = system
@ -407,16 +408,13 @@ private[akka] class ActorCell(
actor.asInstanceOf[InternalActorRef].stop()
}
var currentMessage: Envelope = null
var currentMessage: Envelope = _
var actor: Actor = _
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
@volatile //This must be volatile since it isn't protected by the mailbox status
var mailbox: Mailbox = _
@volatile var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
var nextNameSequence: Long = 0
var watching: Set[ActorRef] = emptyActorRefSet
var watchedBy: Set[ActorRef] = emptyActorRefSet
//Not thread safe, so should only be used inside the actor that inhabits this ActorCell
final protected def randomName(): String = {
@ -428,6 +426,24 @@ private[akka] class ActorCell(
@inline
final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
/**
* INTERNAL API
*
* Returns a reference to the current mailbox
*/
@inline final def mailbox: Mailbox = Unsafe.instance.getObjectVolatile(this, mailboxOffset).asInstanceOf[Mailbox]
/**
* INTERNAL API
*
* replaces the current mailbox using getAndSet semantics
*/
@tailrec final def swapMailbox(newMailbox: Mailbox): Mailbox = {
val oldMailbox = mailbox
if (!Unsafe.instance.compareAndSwapObject(this, mailboxOffset, oldMailbox, newMailbox)) swapMailbox(newMailbox)
else oldMailbox
}
/**
* UntypedActorContext impl
*/
@ -440,7 +456,7 @@ private[akka] class ActorCell(
* Create the mailbox and enqueue the Create() message to ensure that
* this is processed before anything else.
*/
mailbox = dispatcher.createMailbox(this)
swapMailbox(dispatcher.createMailbox(this))
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
mailbox.systemEnqueue(self, Create())
@ -460,16 +476,22 @@ private[akka] class ActorCell(
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
final def stop(): Unit = dispatcher.systemDispatch(this, Terminate())
override final def watch(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Link(subject))
subject
override final def watch(subject: ActorRef): ActorRef = subject match {
case a: InternalActorRef
if (a != self && !watching.contains(a)) {
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
watching += a
}
a
}
override final def unwatch(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Unlink(subject))
subject
override final def unwatch(subject: ActorRef): ActorRef = subject match {
case a: InternalActorRef
if (a != self && watching.contains(a)) {
a.sendSystemMessage(Unwatch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
watching -= a
}
a
}
final def children: Iterable[ActorRef] = childrenRefs.children
@ -477,10 +499,8 @@ private[akka] class ActorCell(
/**
* Impl UntypedActorContext
*/
final def getChildren(): java.lang.Iterable[ActorRef] = {
import scala.collection.JavaConverters.asJavaIterableConverter
asJavaIterableConverter(children).asJava
}
final def getChildren(): java.lang.Iterable[ActorRef] =
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
final def tell(message: Any, sender: ActorRef): Unit =
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)(system))
@ -567,15 +587,35 @@ private[akka] class ActorCell(
def resume(): Unit = if (isNormal) dispatcher resume this
def link(subject: ActorRef): Unit = if (!isTerminating) {
if (system.deathWatch.subscribe(self, subject)) {
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + subject))
def addWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
val watcheeSelf = watchee == self
val watcherSelf = watcher == self
if (watcheeSelf && !watcherSelf) {
if (!watchedBy.contains(watcher)) {
watchedBy += watcher
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + watcher))
}
} else if (!watcheeSelf && watcherSelf) {
watch(watchee)
} else {
system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, self)))
}
}
def unlink(subject: ActorRef): Unit = if (!isTerminating) {
if (system.deathWatch.unsubscribe(self, subject)) {
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + subject))
def remWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
val watcheeSelf = watchee == self
val watcherSelf = watcher == self
if (watcheeSelf && !watcherSelf) {
if (watchedBy.contains(watcher)) {
watchedBy -= watcher
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + watcher))
}
} else if (!watcheeSelf && watcherSelf) {
unwatch(watchee)
} else {
system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, self)))
}
}
@ -605,8 +645,8 @@ private[akka] class ActorCell(
message match {
case Create() create()
case Recreate(cause) recreate(cause)
case Link(subject) link(subject)
case Unlink(subject) unlink(subject)
case Watch(watchee, watcher) addWatcher(watchee, watcher)
case Unwatch(watchee, watcher) remWatcher(watchee, watcher)
case Suspend() suspend()
case Resume() resume()
case Terminate() terminate()
@ -698,11 +738,32 @@ private[akka] class ActorCell(
} finally {
try {
parent.sendSystemMessage(ChildTerminated(self))
system.deathWatch.publish(Terminated(self))
if (!watchedBy.isEmpty) {
val terminated = Terminated(self)(existenceConfirmed = true)
try {
watchedBy foreach {
watcher
try watcher.tell(terminated, self) catch {
case NonFatal(t) system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch"))
}
}
} finally watchedBy = emptyActorRefSet
}
if (!watching.isEmpty) {
try {
watching foreach { // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
case watchee: InternalActorRef try watchee.sendSystemMessage(Unwatch(watchee, self)) catch {
case NonFatal(t) system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch"))
}
}
} finally watching = emptyActorRefSet
}
if (system.settings.DebugLifecycle)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped"))
system.eventStream.publish(Debug(self.path.toString, clazz(a), "stopped"))
} finally {
behaviorStack = ActorCell.behaviorStackPlaceHolder
behaviorStack = behaviorStackPlaceHolder
clearActorFields(a)
actor = null
}

View file

@ -409,16 +409,26 @@ private[akka] object DeadLetterActorRef {
*
* INTERNAL API
*/
private[akka] class EmptyLocalActorRef(
override val provider: ActorRefProvider,
private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
override val path: ActorPath,
val eventStream: EventStream) extends MinimalActorRef {
override def isTerminated(): Boolean = true
override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message)
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
case d: DeadLetter // do NOT form endless loops, since deadLetters will resend!
case _ eventStream.publish(DeadLetter(message, sender, this))
case d: DeadLetter specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend!
case _ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
}
protected def specialHandle(msg: Any): Boolean = msg match {
case w: Watch
if (w.watchee == this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false)
true
case _: Unwatch true // Just ignore
case _ false
}
}
@ -428,12 +438,22 @@ private[akka] class EmptyLocalActorRef(
*
* INTERNAL API
*/
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream)
extends EmptyLocalActorRef(_provider, _path, _eventStream) {
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider,
_path: ActorPath,
_eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) {
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
case d: DeadLetter eventStream.publish(d)
case _ eventStream.publish(DeadLetter(message, sender, this))
case d: DeadLetter if (!specialHandle(d.message)) eventStream.publish(d)
case _ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
}
override protected def specialHandle(msg: Any): Boolean = msg match {
case w: Watch
if (w.watchee != this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false)
true
case w: Unwatch true // Just ignore
case _ false
}
@throws(classOf[java.io.ObjectStreamException])

View file

@ -8,8 +8,8 @@ import java.util.concurrent.atomic.AtomicLong
import akka.dispatch._
import akka.routing._
import akka.AkkaException
import akka.util.{ Switch, Helpers }
import akka.event._
import akka.util.{ NonFatal, Switch, Helpers }
/**
* Interface for all ActorRef providers to implement.
@ -38,11 +38,6 @@ trait ActorRefProvider {
*/
def deadLetters: ActorRef
/**
* Reference to the death watch service.
*/
def deathWatch: DeathWatch
/**
* The root path for all actors within this actor system, including remote
* address if enabled.
@ -162,8 +157,9 @@ trait ActorRefFactory {
* INTERNAL USE ONLY
*/
protected def provider: ActorRefProvider
/**
* Returns the default MessageDispatcher used by this ActorRefFactory
* Returns the default MessageDispatcher associated with this ActorRefFactory
*/
implicit def dispatcher: MessageDispatcher
@ -339,8 +335,6 @@ class LocalActorRefProvider(
override val deadLetters: InternalActorRef = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
override val deathWatch: DeathWatch = new LocalDeathWatch(1024) //TODO make configrable
/*
* generate name for temporary actor refs
*/
@ -379,8 +373,8 @@ class LocalActorRefProvider(
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
message match {
case Supervise(child) // TODO register child in some map to keep track of it and enable shutdown after all dead
case ChildTerminated(child) stop()
case Supervise(_) // TODO register child in some map to keep track of it and enable shutdown after all dead
case ChildTerminated(_) stop()
case _ log.error(this + " received unexpected system message [" + message + "]")
}
}
@ -409,8 +403,8 @@ class LocalActorRefProvider(
def receive = {
case Terminated(_) context.stop(self)
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case NonFatal(e) Status.Failure(e) })
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case NonFatal(e) Status.Failure(e) })
case StopChild(child) context.stop(child); sender ! "ok"
case m deadLetters ! DeadLetter(m, sender, self)
}
@ -441,8 +435,8 @@ class LocalActorRefProvider(
def receive = {
case Terminated(_) eventStream.stopDefaultLoggers(); context.stop(self)
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case NonFatal(e) Status.Failure(e) })
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case NonFatal(e) Status.Failure(e) })
case StopChild(child) context.stop(child); sender ! "ok"
case m deadLetters ! DeadLetter(m, sender, self)
}
@ -480,18 +474,10 @@ class LocalActorRefProvider(
lazy val rootGuardian: InternalActorRef =
new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) {
object Extra {
def unapply(s: String): Option[InternalActorRef] = extraNames.get(s)
}
override def getParent: InternalActorRef = this
override def getSingleChild(name: String): InternalActorRef = {
name match {
override def getSingleChild(name: String): InternalActorRef = name match {
case "temp" tempContainer
case Extra(e) e
case _ super.getSingleChild(name)
}
case other extraNames.get(other).getOrElse(super.getSingleChild(other))
}
}
@ -516,8 +502,8 @@ class LocalActorRefProvider(
def init(_system: ActorSystemImpl) {
system = _system
// chain death watchers so that killing guardian stops the application
deathWatch.subscribe(systemGuardian, guardian)
deathWatch.subscribe(rootGuardian, systemGuardian)
systemGuardian.sendSystemMessage(Watch(guardian, systemGuardian))
rootGuardian.sendSystemMessage(Watch(systemGuardian, rootGuardian))
eventStream.startDefaultLoggers(_system)
}
@ -566,19 +552,3 @@ class LocalActorRefProvider(
def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None
}
class LocalDeathWatch(val mapSize: Int) extends DeathWatch with ActorClassification {
override def publish(event: Event): Unit = {
val monitors = dissociate(classify(event))
if (monitors.nonEmpty) monitors.foreach(_ ! event)
}
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = {
if (!super.subscribe(subscriber, to)) {
subscriber ! Terminated(to)
false
} else true
}
}

View file

@ -408,11 +408,6 @@ abstract class ExtendedActorSystem extends ActorSystem {
*/
def systemGuardian: InternalActorRef
/**
* Implementation of the mechanism which is used for watch()/unwatch().
*/
def deathWatch: DeathWatch
/**
* A ThreadFactory that can be used if the transport needs to create any Threads
*/
@ -485,26 +480,17 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
private[akka] def systemActorOf(props: Props, name: String): ActorRef = {
implicit val timeout = settings.CreationTimeout
Await.result(systemGuardian ? CreateChild(props, name), timeout.duration) match {
case ref: ActorRef ref
case ex: Exception throw ex
}
Await.result((systemGuardian ? CreateChild(props, name)).mapTo[ActorRef], timeout.duration)
}
def actorOf(props: Props, name: String): ActorRef = {
implicit val timeout = settings.CreationTimeout
Await.result(guardian ? CreateChild(props, name), timeout.duration) match {
case ref: ActorRef ref
case ex: Exception throw ex
}
Await.result((guardian ? CreateChild(props, name)).mapTo[ActorRef], timeout.duration)
}
def actorOf(props: Props): ActorRef = {
implicit val timeout = settings.CreationTimeout
Await.result(guardian ? CreateRandomNameChild(props), timeout.duration) match {
case ref: ActorRef ref
case ex: Exception throw ex
}
Await.result((guardian ? CreateRandomNameChild(props)).mapTo[ActorRef], timeout.duration)
}
def stop(actor: ActorRef): Unit = {
@ -547,7 +533,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
//FIXME Why do we need this at all?
val deadLetterQueue: MessageQueue = new MessageQueue {
def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
def enqueue(receiver: ActorRef, envelope: Envelope): Unit =
deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver)
def dequeue() = null
def hasMessages = false
def numberOfMessages = 0
@ -556,8 +543,9 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
//FIXME Why do we need this at all?
val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) {
becomeClosed()
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = deadLetters ! DeadLetter(handle, receiver, receiver)
def systemDrain(): SystemMessage = null
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit =
deadLetters ! DeadLetter(handle, receiver, receiver)
def systemDrain(newContents: SystemMessage): SystemMessage = null
def hasSystemMessages = false
}
@ -570,7 +558,6 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
def lookupRoot: InternalActorRef = provider.rootGuardian
def guardian: InternalActorRef = provider.guardian
def systemGuardian: InternalActorRef = provider.systemGuardian
def deathWatch: DeathWatch = provider.deathWatch
def /(actorName: String): ActorPath = guardian.path / actorName
def /(path: Iterable[String]): ActorPath = guardian.path / path

View file

@ -102,11 +102,15 @@ private[akka] case class ChildTerminated(child: ActorRef) extends SystemMessage
/**
* INTERNAL API
*/
private[akka] case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch
private[akka] case class Watch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to establish a DeathWatch
/**
* INTERNAL API
*/
private[akka] case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch
private[akka] case class Unwatch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to tear down a DeathWatch
/**
* INTERNAL API
*/
private[akka] case object NoMessage extends SystemMessage // switched into the mailbox to signal termination
final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () Unit) extends Runnable {
def run(): Unit =
@ -310,16 +314,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
case 0
shutdownSchedule match {
case UNSCHEDULED
if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) {
scheduleShutdownAction()
()
} else ifSensibleToDoSoThenScheduleShutdown()
if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) scheduleShutdownAction()
else ifSensibleToDoSoThenScheduleShutdown()
case SCHEDULED
if (updateShutdownSchedule(SCHEDULED, RESCHEDULED)) ()
else ifSensibleToDoSoThenScheduleShutdown()
case RESCHEDULED ()
case RESCHEDULED
}
case _ ()
case _
}
private def scheduleShutdownAction(): Unit = {
@ -349,9 +351,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
protected[akka] def unregister(actor: ActorCell) {
if (debug) actors.remove(this, actor.self)
addInhabitants(-1)
val mailBox = actor.mailbox
val mailBox = actor.swapMailbox(deadLetterMailbox)
mailBox.becomeClosed() // FIXME reschedule in tell if possible race with cleanUp is detected in order to properly clean up
actor.mailbox = deadLetterMailbox
mailBox.cleanUp()
}
@ -359,7 +360,6 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
@tailrec
final def run() {
shutdownSchedule match {
case UNSCHEDULED ()
case SCHEDULED
try {
if (inhabitants == 0) shutdown() //Warning, racy
@ -369,6 +369,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
case RESCHEDULED
if (updateShutdownSchedule(RESCHEDULED, SCHEDULED)) scheduleShutdownAction()
else run()
case UNSCHEDULED
}
}
}

View file

@ -50,10 +50,9 @@ class BalancingDispatcher(
private class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue {
override def cleanUp(): Unit = {
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
if (hasSystemMessages) {
val dlq = actor.systemImpl.deadLetterMailbox
var message = systemDrain()
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
var message = systemDrain(NoMessage)
while (message ne null) {
// message must be virgin before being able to systemEnqueue again
val next = message.next
@ -63,7 +62,6 @@ class BalancingDispatcher(
}
}
}
}
protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor, messageQueue)

View file

@ -169,6 +169,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
*/
protected final def systemQueueGet: SystemMessage =
Unsafe.instance.getObjectVolatile(this, AbstractMailbox.systemMessageOffset).asInstanceOf[SystemMessage]
protected final def systemQueuePut(_old: SystemMessage, _new: SystemMessage): Boolean =
Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old, _new)
@ -208,14 +209,14 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
}
final def processAllSystemMessages() {
var nextMessage = systemDrain()
var nextMessage = systemDrain(null)
try {
while ((nextMessage ne null) && !isClosed) {
if (debug) println(actor.self + " processing system message " + nextMessage + " with " + actor.childrenRefs)
actor systemInvoke nextMessage
nextMessage = nextMessage.next
// dont ever execute normal message when system message present!
if (nextMessage eq null) nextMessage = systemDrain()
if (nextMessage eq null) nextMessage = systemDrain(null)
}
} catch {
case NonFatal(e)
@ -235,8 +236,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
protected[dispatch] def cleanUp(): Unit =
if (actor ne null) { // actor is null for the deadLetterMailbox
val dlm = actor.systemImpl.deadLetterMailbox
if (hasSystemMessages) {
var message = systemDrain()
var message = systemDrain(NoMessage)
while (message ne null) {
// message must be virgin before being able to systemEnqueue again
val next = message.next
@ -244,7 +244,6 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
dlm.systemEnqueue(actor.self, message)
message = next
}
}
if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run()
messageQueue.cleanUp(actor, actor.systemImpl.deadLetterQueue)
@ -300,7 +299,7 @@ private[akka] trait SystemMessageQueue {
/**
* Dequeue all messages from system queue and return them as single-linked list.
*/
def systemDrain(): SystemMessage
def systemDrain(newContents: SystemMessage): SystemMessage
def hasSystemMessages: Boolean
}
@ -315,6 +314,8 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
assert(message.next eq null)
if (Mailbox.debug) println(actor.self + " having enqueued " + message)
val head = systemQueueGet
if (head == NoMessage) actor.system.deadLetterMailbox.systemEnqueue(receiver, message)
else {
/*
* this write is safely published by the compareAndSet contained within
* systemQueuePut; Intra-Thread Semantics on page 12 of the JSR133 spec
@ -327,14 +328,16 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
systemEnqueue(receiver, message)
}
}
}
@tailrec
final def systemDrain(): SystemMessage = {
final def systemDrain(newContents: SystemMessage): SystemMessage = {
val head = systemQueueGet
if (systemQueuePut(head, null)) SystemMessage.reverse(head) else systemDrain()
if (systemQueuePut(head, newContents)) SystemMessage.reverse(head) else systemDrain(newContents)
}
def hasSystemMessages: Boolean = systemQueueGet ne null
}
/**

View file

@ -29,30 +29,20 @@ object ThreadPoolConfig {
val defaultTimeout: Duration = Duration(60000L, TimeUnit.MILLISECONDS)
val defaultRejectionPolicy: RejectedExecutionHandler = new SaneRejectedExecutionHandler()
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int = {
import scala.math.{ min, max }
min(max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
}
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int =
math.min(math.max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory =
() new ArrayBlockingQueue[Runnable](capacity, fair)
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = () new ArrayBlockingQueue[Runnable](capacity, fair)
def synchronousQueue(fair: Boolean): QueueFactory =
() new SynchronousQueue[Runnable](fair)
def synchronousQueue(fair: Boolean): QueueFactory = () new SynchronousQueue[Runnable](fair)
def linkedBlockingQueue(): QueueFactory =
() new LinkedBlockingQueue[Runnable]()
def linkedBlockingQueue(): QueueFactory = () new LinkedBlockingQueue[Runnable]()
def linkedBlockingQueue(capacity: Int): QueueFactory =
() new LinkedBlockingQueue[Runnable](capacity)
def linkedBlockingQueue(capacity: Int): QueueFactory = () new LinkedBlockingQueue[Runnable](capacity)
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory =
() queue
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = () queue
def reusableQueue(queueFactory: QueueFactory): QueueFactory = {
val queue = queueFactory()
() queue
}
def reusableQueue(queueFactory: QueueFactory): QueueFactory = reusableQueue(queueFactory())
}
/**
@ -157,7 +147,8 @@ case class ThreadPoolConfigBuilder(config: ThreadPoolConfig) {
def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = newQueueFactory))
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder = fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder =
fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
}
object MonitorableThreadFactory {

View file

@ -1,19 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.event
import akka.actor._
/**
* The contract of DeathWatch is not properly expressed using the type system
* Whenever there is a publish, all listeners to the Terminated Actor should be atomically removed
* A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down
* See LocalDeathWatch for semantics
*/
abstract class DeathWatch extends ActorEventBus with ActorClassifier {
type Event = Terminated
protected final def classify(event: Event): Classifier = event.actor
}

View file

@ -4,12 +4,10 @@
package akka.pattern
import java.util.concurrent.TimeoutException
import akka.dispatch.{ Promise, Terminate, SystemMessage, Future }
import akka.event.DeathWatch
import akka.util.Timeout
import annotation.tailrec
import akka.util.Unsafe
import akka.actor._
import akka.dispatch._
import akka.util.{ NonFatal, Timeout, Unsafe }
/**
* This is what is used to complete a Future that is returned from an ask/? call,
@ -164,6 +162,7 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
extends MinimalActorRef {
import PromiseActorRef._
import AbstractPromiseActorRef.stateOffset
import AbstractPromiseActorRef.watchedByOffset
/**
* As an optimization for the common (local) case we only register this PromiseActorRef
@ -180,14 +179,43 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
@volatile
private[this] var _stateDoNotCallMeDirectly: AnyRef = _
@inline
private def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
@volatile
private[this] var _watchedByDoNotCallMeDirectly: Set[ActorRef] = ActorCell.emptyActorRefSet
@inline
private def updateState(oldState: AnyRef, newState: AnyRef): Boolean = Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
private[this] def watchedBy: Set[ActorRef] = Unsafe.instance.getObjectVolatile(this, watchedByOffset).asInstanceOf[Set[ActorRef]]
@inline
private def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
private[this] def updateWatchedBy(oldWatchedBy: Set[ActorRef], newWatchedBy: Set[ActorRef]): Boolean =
Unsafe.instance.compareAndSwapObject(this, watchedByOffset, oldWatchedBy, newWatchedBy)
@tailrec // Returns false if the Promise is already completed
private[this] final def addWatcher(watcher: ActorRef): Boolean = watchedBy match {
case null false
case other updateWatchedBy(other, other + watcher) || addWatcher(watcher)
}
@tailrec
private[this] final def remWatcher(watcher: ActorRef): Unit = watchedBy match {
case null ()
case other if (!updateWatchedBy(other, other - watcher)) remWatcher(watcher)
}
@tailrec
private[this] final def clearWatchers(): Set[ActorRef] = watchedBy match {
case null ActorCell.emptyActorRefSet
case other if (!updateWatchedBy(other, null)) clearWatchers() else other
}
@inline
private[this] def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
@inline
private[this] def updateState(oldState: AnyRef, newState: AnyRef): Boolean =
Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
@inline
private[this] def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
override def getParent: InternalActorRef = provider.tempContainer
@ -218,19 +246,24 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
override def !(message: Any)(implicit sender: ActorRef = null): Unit = state match {
case Stopped | _: StoppedWithPath provider.deadLetters ! message
case _
val completedJustNow = result.tryComplete {
case _ if (!(result.tryComplete {
message match {
case Status.Success(r) Right(r)
case Status.Failure(f) Left(f)
case other Right(other)
}
}
if (!completedJustNow) provider.deadLetters ! message
})) provider.deadLetters ! message
}
override def sendSystemMessage(message: SystemMessage): Unit = message match {
case _: Terminate stop()
case Watch(watchee, watcher)
if (watchee == this && watcher != this) {
if (!addWatcher(watcher)) watcher ! Terminated(watchee)(existenceConfirmed = true)
} else System.err.println("BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, this))
case Unwatch(watchee, watcher)
if (watchee == this && watcher != this) remWatcher(watcher)
else System.err.println("BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, this))
case _
}
@ -241,23 +274,20 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
@tailrec
override def stop(): Unit = {
def ensurePromiseCompleted(): Unit =
def ensureCompleted(): Unit = {
if (!result.isCompleted) result.tryComplete(Left(new ActorKilledException("Stopped")))
state match {
case null
// if path was never queried nobody can possibly be watching us, so we don't have to publish termination either
if (updateState(null, Stopped)) ensurePromiseCompleted()
else stop()
case p: ActorPath
if (updateState(p, StoppedWithPath(p))) {
try {
ensurePromiseCompleted()
provider.deathWatch.publish(Terminated(this))
} finally {
provider.unregisterTempActor(p)
val watchers = clearWatchers()
if (!watchers.isEmpty) {
val termination = Terminated(this)(existenceConfirmed = true)
watchers foreach { w try w.tell(termination, this) catch { case NonFatal(t) /* FIXME LOG THIS */ } }
}
} else stop()
case Stopped | _: StoppedWithPath
}
state match {
case null // if path was never queried nobody can possibly be watching us, so we don't have to publish termination either
if (updateState(null, Stopped)) ensureCompleted() else stop()
case p: ActorPath
if (updateState(p, StoppedWithPath(p))) { try ensureCompleted() finally provider.unregisterTempActor(p) } else stop()
case Stopped | _: StoppedWithPath // already stopped
case Registering stop() // spin until registration is completed before stopping
}
}

View file

@ -4,9 +4,9 @@
package akka.pattern
import akka.dispatch.{ Promise, Future }
import akka.actor._
import akka.util.{ Timeout, Duration }
import akka.dispatch.{ Unwatch, Watch, Promise, Future }
trait GracefulStopSupport {
/**
@ -34,18 +34,21 @@ trait GracefulStopSupport {
* is completed with failure [[akka.pattern.AskTimeoutException]].
*/
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = {
if (target.isTerminated) {
Promise.successful(true)
} else system match {
if (target.isTerminated) Promise.successful(true)
else system match {
case e: ExtendedActorSystem
val internalTarget = target.asInstanceOf[InternalActorRef]
val ref = PromiseActorRef(e.provider, Timeout(timeout))
e.deathWatch.subscribe(ref, target)
ref.result onComplete {
case Right(Terminated(`target`)) () // Ignore
case _ e.deathWatch.unsubscribe(ref, target)
} // Just making sure we're not leaking here
internalTarget.sendSystemMessage(Watch(target, ref))
ref.result onComplete { // Just making sure we're not leaking here
case Right(Terminated(`target`)) ()
case _ internalTarget.sendSystemMessage(Unwatch(target, ref))
}
target ! PoisonPill
ref.result map { case Terminated(`target`) true }
ref.result map {
case Terminated(`target`) true
case _ false
}
case s throw new IllegalArgumentException("Unknown ActorSystem implementation: '" + s + "'")
}
}

View file

@ -182,7 +182,7 @@ public class FaultHandlingTestBase {
final TestProbe probe = new TestProbe(system);
probe.watch(child);
child.tell(new IllegalArgumentException());
probe.expectMsg(new Terminated(child));
probe.expectMsg(new Terminated(child, true));
//#stop
//#escalate-kill
@ -190,7 +190,7 @@ public class FaultHandlingTestBase {
probe.watch(child);
assert Await.result(ask(child, "get", 5000), timeout).equals(0);
child.tell(new Exception());
probe.expectMsg(new Terminated(child));
probe.expectMsg(new Terminated(child, true));
//#escalate-kill
//#escalate-restart

View file

@ -111,7 +111,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
//#stop
watch(child) // have testActor watch child
child ! new IllegalArgumentException // break it
expectMsg(Terminated(child))
expectMsg(Terminated(child)(existenceConfirmed = true))
child.isTerminated must be(true)
//#stop
}
@ -125,7 +125,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
expectMsg(0)
child2 ! new Exception("CRASH") // escalate failure
expectMsg(Terminated(child2))
expectMsg(Terminated(child2)(existenceConfirmed = true))
//#escalate-kill
//#escalate-restart
val supervisor2 = system.actorOf(Props[Supervisor2], "supervisor2")

View file

@ -6351,605 +6351,6 @@ public final class RemoteProtocol {
// @@protoc_insertion_point(class_scope:DeployProtocol)
}
public interface DaemonMsgWatchProtocolOrBuilder
extends com.google.protobuf.MessageOrBuilder {
// required .ActorRefProtocol watcher = 1;
boolean hasWatcher();
akka.remote.RemoteProtocol.ActorRefProtocol getWatcher();
akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatcherOrBuilder();
// required .ActorRefProtocol watched = 2;
boolean hasWatched();
akka.remote.RemoteProtocol.ActorRefProtocol getWatched();
akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatchedOrBuilder();
}
public static final class DaemonMsgWatchProtocol extends
com.google.protobuf.GeneratedMessage
implements DaemonMsgWatchProtocolOrBuilder {
// Use DaemonMsgWatchProtocol.newBuilder() to construct.
private DaemonMsgWatchProtocol(Builder builder) {
super(builder);
}
private DaemonMsgWatchProtocol(boolean noInit) {}
private static final DaemonMsgWatchProtocol defaultInstance;
public static DaemonMsgWatchProtocol getDefaultInstance() {
return defaultInstance;
}
public DaemonMsgWatchProtocol getDefaultInstanceForType() {
return defaultInstance;
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_fieldAccessorTable;
}
private int bitField0_;
// required .ActorRefProtocol watcher = 1;
public static final int WATCHER_FIELD_NUMBER = 1;
private akka.remote.RemoteProtocol.ActorRefProtocol watcher_;
public boolean hasWatcher() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public akka.remote.RemoteProtocol.ActorRefProtocol getWatcher() {
return watcher_;
}
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatcherOrBuilder() {
return watcher_;
}
// required .ActorRefProtocol watched = 2;
public static final int WATCHED_FIELD_NUMBER = 2;
private akka.remote.RemoteProtocol.ActorRefProtocol watched_;
public boolean hasWatched() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public akka.remote.RemoteProtocol.ActorRefProtocol getWatched() {
return watched_;
}
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatchedOrBuilder() {
return watched_;
}
private void initFields() {
watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) return isInitialized == 1;
if (!hasWatcher()) {
memoizedIsInitialized = 0;
return false;
}
if (!hasWatched()) {
memoizedIsInitialized = 0;
return false;
}
if (!getWatcher().isInitialized()) {
memoizedIsInitialized = 0;
return false;
}
if (!getWatched().isInitialized()) {
memoizedIsInitialized = 0;
return false;
}
memoizedIsInitialized = 1;
return true;
}
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeMessage(1, watcher_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
output.writeMessage(2, watched_);
}
getUnknownFields().writeTo(output);
}
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
size = 0;
if (((bitField0_ & 0x00000001) == 0x00000001)) {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, watcher_);
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(2, watched_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(java.io.InputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input)) {
return builder.buildParsed();
} else {
return null;
}
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseDelimitedFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
return builder.buildParsed();
} else {
return null;
}
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(akka.remote.RemoteProtocol.DaemonMsgWatchProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
@java.lang.Override
protected Builder newBuilderForType(
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
Builder builder = new Builder(parent);
return builder;
}
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder>
implements akka.remote.RemoteProtocol.DaemonMsgWatchProtocolOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_fieldAccessorTable;
}
// Construct using akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
super(parent);
maybeForceBuilderInitialization();
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
getWatcherFieldBuilder();
getWatchedFieldBuilder();
}
}
private static Builder create() {
return new Builder();
}
public Builder clear() {
super.clear();
if (watcherBuilder_ == null) {
watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
} else {
watcherBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
if (watchedBuilder_ == null) {
watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
} else {
watchedBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
public Builder clone() {
return create().mergeFrom(buildPartial());
}
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.getDescriptor();
}
public akka.remote.RemoteProtocol.DaemonMsgWatchProtocol getDefaultInstanceForType() {
return akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.getDefaultInstance();
}
public akka.remote.RemoteProtocol.DaemonMsgWatchProtocol build() {
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
private akka.remote.RemoteProtocol.DaemonMsgWatchProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(
result).asInvalidProtocolBufferException();
}
return result;
}
public akka.remote.RemoteProtocol.DaemonMsgWatchProtocol buildPartial() {
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol result = new akka.remote.RemoteProtocol.DaemonMsgWatchProtocol(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
to_bitField0_ |= 0x00000001;
}
if (watcherBuilder_ == null) {
result.watcher_ = watcher_;
} else {
result.watcher_ = watcherBuilder_.build();
}
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
if (watchedBuilder_ == null) {
result.watched_ = watched_;
} else {
result.watched_ = watchedBuilder_.build();
}
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof akka.remote.RemoteProtocol.DaemonMsgWatchProtocol) {
return mergeFrom((akka.remote.RemoteProtocol.DaemonMsgWatchProtocol)other);
} else {
super.mergeFrom(other);
return this;
}
}
public Builder mergeFrom(akka.remote.RemoteProtocol.DaemonMsgWatchProtocol other) {
if (other == akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.getDefaultInstance()) return this;
if (other.hasWatcher()) {
mergeWatcher(other.getWatcher());
}
if (other.hasWatched()) {
mergeWatched(other.getWatched());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
public final boolean isInitialized() {
if (!hasWatcher()) {
return false;
}
if (!hasWatched()) {
return false;
}
if (!getWatcher().isInitialized()) {
return false;
}
if (!getWatched().isInitialized()) {
return false;
}
return true;
}
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder(
this.getUnknownFields());
while (true) {
int tag = input.readTag();
switch (tag) {
case 0:
this.setUnknownFields(unknownFields.build());
onChanged();
return this;
default: {
if (!parseUnknownField(input, unknownFields,
extensionRegistry, tag)) {
this.setUnknownFields(unknownFields.build());
onChanged();
return this;
}
break;
}
case 10: {
akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder();
if (hasWatcher()) {
subBuilder.mergeFrom(getWatcher());
}
input.readMessage(subBuilder, extensionRegistry);
setWatcher(subBuilder.buildPartial());
break;
}
case 18: {
akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder();
if (hasWatched()) {
subBuilder.mergeFrom(getWatched());
}
input.readMessage(subBuilder, extensionRegistry);
setWatched(subBuilder.buildPartial());
break;
}
}
}
}
private int bitField0_;
// required .ActorRefProtocol watcher = 1;
private akka.remote.RemoteProtocol.ActorRefProtocol watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> watcherBuilder_;
public boolean hasWatcher() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public akka.remote.RemoteProtocol.ActorRefProtocol getWatcher() {
if (watcherBuilder_ == null) {
return watcher_;
} else {
return watcherBuilder_.getMessage();
}
}
public Builder setWatcher(akka.remote.RemoteProtocol.ActorRefProtocol value) {
if (watcherBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
watcher_ = value;
onChanged();
} else {
watcherBuilder_.setMessage(value);
}
bitField0_ |= 0x00000001;
return this;
}
public Builder setWatcher(
akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) {
if (watcherBuilder_ == null) {
watcher_ = builderForValue.build();
onChanged();
} else {
watcherBuilder_.setMessage(builderForValue.build());
}
bitField0_ |= 0x00000001;
return this;
}
public Builder mergeWatcher(akka.remote.RemoteProtocol.ActorRefProtocol value) {
if (watcherBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001) &&
watcher_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) {
watcher_ =
akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(watcher_).mergeFrom(value).buildPartial();
} else {
watcher_ = value;
}
onChanged();
} else {
watcherBuilder_.mergeFrom(value);
}
bitField0_ |= 0x00000001;
return this;
}
public Builder clearWatcher() {
if (watcherBuilder_ == null) {
watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
onChanged();
} else {
watcherBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getWatcherBuilder() {
bitField0_ |= 0x00000001;
onChanged();
return getWatcherFieldBuilder().getBuilder();
}
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatcherOrBuilder() {
if (watcherBuilder_ != null) {
return watcherBuilder_.getMessageOrBuilder();
} else {
return watcher_;
}
}
private com.google.protobuf.SingleFieldBuilder<
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>
getWatcherFieldBuilder() {
if (watcherBuilder_ == null) {
watcherBuilder_ = new com.google.protobuf.SingleFieldBuilder<
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>(
watcher_,
getParentForChildren(),
isClean());
watcher_ = null;
}
return watcherBuilder_;
}
// required .ActorRefProtocol watched = 2;
private akka.remote.RemoteProtocol.ActorRefProtocol watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> watchedBuilder_;
public boolean hasWatched() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
public akka.remote.RemoteProtocol.ActorRefProtocol getWatched() {
if (watchedBuilder_ == null) {
return watched_;
} else {
return watchedBuilder_.getMessage();
}
}
public Builder setWatched(akka.remote.RemoteProtocol.ActorRefProtocol value) {
if (watchedBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
watched_ = value;
onChanged();
} else {
watchedBuilder_.setMessage(value);
}
bitField0_ |= 0x00000002;
return this;
}
public Builder setWatched(
akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) {
if (watchedBuilder_ == null) {
watched_ = builderForValue.build();
onChanged();
} else {
watchedBuilder_.setMessage(builderForValue.build());
}
bitField0_ |= 0x00000002;
return this;
}
public Builder mergeWatched(akka.remote.RemoteProtocol.ActorRefProtocol value) {
if (watchedBuilder_ == null) {
if (((bitField0_ & 0x00000002) == 0x00000002) &&
watched_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) {
watched_ =
akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(watched_).mergeFrom(value).buildPartial();
} else {
watched_ = value;
}
onChanged();
} else {
watchedBuilder_.mergeFrom(value);
}
bitField0_ |= 0x00000002;
return this;
}
public Builder clearWatched() {
if (watchedBuilder_ == null) {
watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
onChanged();
} else {
watchedBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getWatchedBuilder() {
bitField0_ |= 0x00000002;
onChanged();
return getWatchedFieldBuilder().getBuilder();
}
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatchedOrBuilder() {
if (watchedBuilder_ != null) {
return watchedBuilder_.getMessageOrBuilder();
} else {
return watched_;
}
}
private com.google.protobuf.SingleFieldBuilder<
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>
getWatchedFieldBuilder() {
if (watchedBuilder_ == null) {
watchedBuilder_ = new com.google.protobuf.SingleFieldBuilder<
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>(
watched_,
getParentForChildren(),
isClean());
watched_ = null;
}
return watchedBuilder_;
}
// @@protoc_insertion_point(builder_scope:DaemonMsgWatchProtocol)
}
static {
defaultInstance = new DaemonMsgWatchProtocol(true);
defaultInstance.initFields();
}
// @@protoc_insertion_point(class_scope:DaemonMsgWatchProtocol)
}
private static com.google.protobuf.Descriptors.Descriptor
internal_static_AkkaRemoteProtocol_descriptor;
private static
@ -7000,11 +6401,6 @@ public final class RemoteProtocol {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_DeployProtocol_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_DaemonMsgWatchProtocol_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_DaemonMsgWatchProtocol_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
@ -7039,11 +6435,9 @@ public final class RemoteProtocol {
"ssCreator\030\003 \001(\t\022\017\n\007creator\030\004 \001(\014\022\024\n\014rout" +
"erConfig\030\005 \001(\014\"S\n\016DeployProtocol\022\014\n\004path" +
"\030\001 \002(\t\022\016\n\006config\030\002 \001(\014\022\024\n\014routerConfig\030\003" +
" \001(\014\022\r\n\005scope\030\004 \001(\014\"`\n\026DaemonMsgWatchPro" +
"tocol\022\"\n\007watcher\030\001 \002(\0132\021.ActorRefProtoco" +
"l\022\"\n\007watched\030\002 \002(\0132\021.ActorRefProtocol*7\n" +
"\013CommandType\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOWN\020\002\022" +
"\r\n\tHEARTBEAT\020\003B\017\n\013akka.remoteH\001"
" \001(\014\022\r\n\005scope\030\004 \001(\014*7\n\013CommandType\022\013\n\007CO" +
"NNECT\020\001\022\014\n\010SHUTDOWN\020\002\022\r\n\tHEARTBEAT\020\003B\017\n\013" +
"akka.remoteH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -7130,14 +6524,6 @@ public final class RemoteProtocol {
new java.lang.String[] { "Path", "Config", "RouterConfig", "Scope", },
akka.remote.RemoteProtocol.DeployProtocol.class,
akka.remote.RemoteProtocol.DeployProtocol.Builder.class);
internal_static_DaemonMsgWatchProtocol_descriptor =
getDescriptor().getMessageTypes().get(10);
internal_static_DaemonMsgWatchProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_DaemonMsgWatchProtocol_descriptor,
new java.lang.String[] { "Watcher", "Watched", },
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.class,
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.Builder.class);
return null;
}
};

View file

@ -108,11 +108,3 @@ message DeployProtocol {
optional bytes routerConfig = 3;
optional bytes scope = 4;
}
/**
* Serialization of akka.remote.DaemonMsgWatch
*/
message DaemonMsgWatchProtocol {
required ActorRefProtocol watcher = 1;
required ActorRefProtocol watched = 2;
}

View file

@ -15,7 +15,6 @@ akka {
serializers {
proto = "akka.serialization.ProtobufSerializer"
daemon-create = "akka.serialization.DaemonMsgCreateSerializer"
daemon-watch = "akka.serialization.DaemonMsgWatchSerializer"
}
@ -24,7 +23,6 @@ akka {
# does, need to use the more specific one here in order to avoid ambiguity
"com.google.protobuf.GeneratedMessage" = proto
"akka.remote.DaemonMsgCreate" = daemon-create
"akka.remote.DaemonMsgWatch" = daemon-watch
}
deployment {

View file

@ -6,7 +6,7 @@ package akka.remote
import akka.actor._
import akka.dispatch._
import akka.event.{ DeathWatch, Logging, LoggingAdapter }
import akka.event.{ Logging, LoggingAdapter }
import akka.event.EventStream
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
@ -34,8 +34,6 @@ private[akka] class RemoteActorRefProvider(
override def rootPath: ActorPath = local.rootPath
override def deadLetters: InternalActorRef = local.deadLetters
override val deathWatch: DeathWatch = new RemoteDeathWatch(local.deathWatch, this)
// these are only available after init()
override def rootGuardian: InternalActorRef = local.rootGuardian
override def guardian: InternalActorRef = local.guardian
@ -247,24 +245,3 @@ private[akka] class RemoteActorRef private[akka] (
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = SerializedActorRef(path)
}
private[akka] class RemoteDeathWatch(val local: DeathWatch, val provider: RemoteActorRefProvider) extends DeathWatch {
override def subscribe(watcher: ActorRef, watched: ActorRef): Boolean = watched match {
case r: RemoteRef
val ret = local.subscribe(watcher, watched)
provider.actorFor(r.path.root / "remote") ! DaemonMsgWatch(watcher, watched)
ret
case l: LocalRef
local.subscribe(watcher, watched)
case _
provider.log.error("unknown ActorRef type {} as DeathWatch target", watched.getClass)
false
}
override def unsubscribe(watcher: ActorRef, watched: ActorRef): Boolean = local.unsubscribe(watcher, watched)
override def unsubscribe(watcher: ActorRef): Unit = local.unsubscribe(watcher)
override def publish(event: Terminated): Unit = local.publish(event)
}

View file

@ -8,10 +8,10 @@ import scala.annotation.tailrec
import akka.actor.{ VirtualPathContainer, Terminated, Deploy, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor }
import akka.event.LoggingAdapter
import akka.dispatch.Watch
private[akka] sealed trait DaemonMsg
private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: String, supervisor: ActorRef) extends DaemonMsg
private[akka] case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg
/**
* Internal system "daemon" actor for remote internal communication.
@ -62,18 +62,15 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath
val actor = system.provider.actorOf(system, props, supervisor.asInstanceOf[InternalActorRef],
path, false, Some(deploy), true)
addChild(subpath.mkString("/"), actor)
system.deathWatch.subscribe(this, actor)
this.sendSystemMessage(Watch(actor, this))
case _
log.error("remote path does not match path from message [{}]", message)
}
case DaemonMsgWatch(watcher, watched)
val other = system.actorFor(watcher.path.root / "remote")
system.deathWatch.subscribe(other, watched)
}
case Terminated(child: LocalActorRef) removeChild(child.path.elements.drop(1).mkString("/"))
case t: Terminated system.deathWatch.publish(t)
case t: Terminated
case unknown log.warning("Unknown message {} received by {}", unknown, this)
}

View file

@ -287,8 +287,7 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re
case l: LocalRef
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
remoteMessage.payload match {
case _: SystemMessage if useUntrustedMode log.warning("operating in UntrustedMode, dropping inbound system message")
case _: AutoReceivedMessage if useUntrustedMode log.warning("operating in UntrustedMode, dropping inbound AutoReceivedMessage")
case msg: PossiblyHarmful if useUntrustedMode log.warning("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
case msg: SystemMessage l.sendSystemMessage(msg)
case msg l.!(msg)(remoteMessage.sender)
}

View file

@ -1,43 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.serialization
import akka.actor.ActorRef
import akka.remote.DaemonMsgWatch
import akka.remote.RemoteProtocol.ActorRefProtocol
import akka.remote.RemoteProtocol.DaemonMsgWatchProtocol
import akka.actor.ExtendedActorSystem
/**
* Serializes akka's internal DaemonMsgWatch using protobuf.
*
* INTERNAL API
*/
private[akka] class DaemonMsgWatchSerializer(val system: ExtendedActorSystem) extends Serializer {
import ProtobufSerializer.serializeActorRef
import ProtobufSerializer.deserializeActorRef
def includeManifest: Boolean = false
def identifier = 4
def toBinary(obj: AnyRef): Array[Byte] = obj match {
case DaemonMsgWatch(watcher, watched)
DaemonMsgWatchProtocol.newBuilder.
setWatcher(serializeActorRef(watcher)).
setWatched(serializeActorRef(watched)).
build.toByteArray
case _
throw new IllegalArgumentException(
"Can't serialize a non-DaemonMsgWatch message using DaemonMsgWatchSerializer [%s]".format(obj))
}
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
val proto = DaemonMsgWatchProtocol.parseFrom(bytes)
DaemonMsgWatch(
watcher = deserializeActorRef(system, proto.getWatcher),
watched = deserializeActorRef(system, proto.getWatched))
}
}

View file

@ -1,49 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.serialization
import akka.testkit.AkkaSpec
import akka.remote.DaemonMsgWatch
import akka.actor.Actor
import akka.actor.Props
object DaemonMsgWatchSerializerSpec {
class MyActor extends Actor {
def receive = {
case _
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class DaemonMsgWatchSerializerSpec extends AkkaSpec {
import DaemonMsgWatchSerializerSpec._
val ser = SerializationExtension(system)
"Serialization" must {
"resolve DaemonMsgWatchSerializer" in {
ser.serializerFor(classOf[DaemonMsgWatch]).getClass must be(classOf[DaemonMsgWatchSerializer])
}
"serialize and de-serialize DaemonMsgWatch" in {
val watcher = system.actorOf(Props[MyActor], "watcher")
val watched = system.actorOf(Props[MyActor], "watched")
val msg = DaemonMsgWatch(watcher, watched)
val bytes = ser.serialize(msg) match {
case Left(exception) fail(exception)
case Right(bytes) bytes
}
ser.deserialize(bytes.asInstanceOf[Array[Byte]], classOf[DaemonMsgWatch]) match {
case Left(exception) fail(exception)
case Right(m) assert(m === msg)
}
}
}
}

View file

@ -12,7 +12,6 @@ import akka.util.duration._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor.PoisonPill
import akka.actor.CreateChild
import akka.actor.DeadLetter
import java.util.concurrent.TimeoutException
import akka.dispatch.{ Await, MessageDispatcher }
@ -115,9 +114,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
"akka.actor.debug.lifecycle" -> true, "akka.actor.debug.event-stream" -> true,
"akka.loglevel" -> "DEBUG", "akka.stdout-loglevel" -> "DEBUG")
val system = ActorSystem("AkkaSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(AkkaSpec.testConf))
val spec = new AkkaSpec(system) {
val ref = Seq(testActor, system.actorOf(Props.empty, "name"))
}
val spec = new AkkaSpec(system) { val ref = Seq(testActor, system.actorOf(Props.empty, "name")) }
spec.ref foreach (_.isTerminated must not be true)
system.shutdown()
spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds)