Renaming startsMonitoring/stopsMonitoring to startsWatching and stopsWatching

This commit is contained in:
Viktor Klang 2011-11-16 16:46:16 +01:00
parent af3600b175
commit 18bfa26272
15 changed files with 33 additions and 33 deletions

View file

@ -20,7 +20,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
"notify with one Terminated message when an Actor is stopped" in { "notify with one Terminated message when an Actor is stopped" in {
val terminal = actorOf(Props(context { case _ })) val terminal = actorOf(Props(context { case _ }))
testActor startsMonitoring terminal testActor startsWatching terminal
testActor ! "ping" testActor ! "ping"
expectMsg("ping") expectMsg("ping")
@ -67,7 +67,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
monitor2 ! "ping" monitor2 ! "ping"
expectMsg("pong") //Needs to be here since startsMonitoring and stopsMonitoring are asynchronous expectMsg("pong") //Needs to be here since startsWatching and stopsWatching are asynchronous
terminal ! PoisonPill terminal ! PoisonPill
@ -114,8 +114,8 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
})) }))
val failed, brother = (supervisor ? Props.empty).as[ActorRef].get val failed, brother = (supervisor ? Props.empty).as[ActorRef].get
brother startsMonitoring failed brother startsWatching failed
testActor startsMonitoring brother testActor startsWatching brother
failed ! Kill failed ! Kill
val result = receiveWhile(3 seconds, messages = 3) { val result = receiveWhile(3 seconds, messages = 3) {

View file

@ -58,7 +58,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
val forward = actorOf(new Forwarder(testActor)) val forward = actorOf(new Forwarder(testActor))
val fsm = actorOf(new MyFSM(testActor)) val fsm = actorOf(new MyFSM(testActor))
val sup = actorOf(Props(new Actor { val sup = actorOf(Props(new Actor {
self startsMonitoring fsm self startsWatching fsm
def receive = { case _ } def receive = { case _ }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))

View file

@ -159,13 +159,13 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
expectNoMsg(Duration.Zero) expectNoMsg(Duration.Zero)
assert(set == Set(1, 2), set + " was not Set(1, 2)") assert(set == Set(1, 2), set + " was not Set(1, 2)")
supervisor startsMonitoring actor supervisor startsWatching actor
expectMsgPF(hint = "now monitoring") { expectMsgPF(hint = "now monitoring") {
case Logging.Debug(ref, msg: String) case Logging.Debug(ref, msg: String)
ref == supervisor.underlyingActor && msg.startsWith("now monitoring") ref == supervisor.underlyingActor && msg.startsWith("now monitoring")
} }
supervisor stopsMonitoring actor supervisor stopsWatching actor
expectMsgPF(hint = "stopped monitoring") { expectMsgPF(hint = "stopped monitoring") {
case Logging.Debug(ref, msg: String) case Logging.Debug(ref, msg: String)
ref == supervisor.underlyingActor && msg.startsWith("stopped monitoring") ref == supervisor.underlyingActor && msg.startsWith("stopped monitoring")

View file

@ -228,7 +228,7 @@ class RestartStrategySpec extends AkkaSpec {
}) })
val slave = (boss ? slaveProps).as[ActorRef].get val slave = (boss ? slaveProps).as[ActorRef].get
boss startsMonitoring slave boss startsWatching slave
slave ! Ping slave ! Ping
slave ! Crash slave ! Crash

View file

@ -53,7 +53,7 @@ class SupervisorHierarchySpec extends AkkaSpec {
val countDownMax = new CountDownLatch(1) val countDownMax = new CountDownLatch(1)
val boss = actorOf(Props(new Actor { val boss = actorOf(Props(new Actor {
val crasher = context.actorOf(Props(new CountDownActor(countDownMessages))) val crasher = context.actorOf(Props(new CountDownActor(countDownMessages)))
self startsMonitoring crasher self startsWatching crasher
protected def receive = { protected def receive = {
case "killCrasher" crasher ! Kill case "killCrasher" crasher ! Kill

View file

@ -345,7 +345,7 @@ abstract class ActorModelSpec extends AkkaSpec {
val stopLatch = new CountDownLatch(num) val stopLatch = new CountDownLatch(num)
val waitTime = (30 seconds).dilated.toMillis val waitTime = (30 seconds).dilated.toMillis
val boss = actorOf(Props(context { val boss = actorOf(Props(context {
case "run" for (_ 1 to num) (context.self startsMonitoring context.actorOf(props)) ! cachedMessage case "run" for (_ 1 to num) (context.self startsWatching context.actorOf(props)) ! cachedMessage
case Terminated(child) stopLatch.countDown() case Terminated(child) stopLatch.countDown()
}).withDispatcher(wavesSupervisorDispatcher(dispatcher))) }).withDispatcher(wavesSupervisorDispatcher(dispatcher)))
boss ! "run" boss ! "run"

View file

@ -376,13 +376,13 @@ trait Actor {
* Registers this actor as a Monitor for the provided ActorRef * Registers this actor as a Monitor for the provided ActorRef
* @return the provided ActorRef * @return the provided ActorRef
*/ */
def watch(subject: ActorRef): ActorRef = self startsMonitoring subject def watch(subject: ActorRef): ActorRef = self startsWatching subject
/** /**
* Unregisters this actor as Monitor for the provided ActorRef * Unregisters this actor as Monitor for the provided ActorRef
* @return the provided ActorRef * @return the provided ActorRef
*/ */
def unwatch(subject: ActorRef): ActorRef = self stopsMonitoring subject def unwatch(subject: ActorRef): ActorRef = self stopsWatching subject
// ========================================= // =========================================
// ==== INTERNAL IMPLEMENTATION DETAILS ==== // ==== INTERNAL IMPLEMENTATION DETAILS ====

View file

@ -118,13 +118,13 @@ private[akka] class ActorCell(
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
private[akka] def stop(): Unit = dispatcher.systemDispatch(this, Terminate()) private[akka] def stop(): Unit = dispatcher.systemDispatch(this, Terminate())
final def startsMonitoring(subject: ActorRef): ActorRef = { final def startsWatching(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Link(subject)) dispatcher.systemDispatch(this, Link(subject))
subject subject
} }
final def stopsMonitoring(subject: ActorRef): ActorRef = { final def stopsWatching(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Unlink(subject)) dispatcher.systemDispatch(this, Unlink(subject))
subject subject

View file

@ -133,7 +133,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
* *
* @return the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def startsMonitoring(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS def startsWatching(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS
/** /**
* Deregisters this actor from being a death monitor of the provided ActorRef * Deregisters this actor from being a death monitor of the provided ActorRef
@ -142,7 +142,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
* *
* @return the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def stopsMonitoring(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS def stopsWatching(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS
override def hashCode: Int = HashCode.hash(HashCode.SEED, address) override def hashCode: Int = HashCode.hash(HashCode.SEED, address)
@ -212,7 +212,7 @@ class LocalActorRef private[akka] (
* *
* @return the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def startsMonitoring(subject: ActorRef): ActorRef = actorCell.startsMonitoring(subject) def startsWatching(subject: ActorRef): ActorRef = actorCell.startsWatching(subject)
/** /**
* Deregisters this actor from being a death monitor of the provided ActorRef * Deregisters this actor from being a death monitor of the provided ActorRef
@ -221,7 +221,7 @@ class LocalActorRef private[akka] (
* *
* @return the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def stopsMonitoring(subject: ActorRef): ActorRef = actorCell.stopsMonitoring(subject) def stopsWatching(subject: ActorRef): ActorRef = actorCell.stopsWatching(subject)
// ========= AKKA PROTECTED FUNCTIONS ========= // ========= AKKA PROTECTED FUNCTIONS =========
@ -319,9 +319,9 @@ trait UnsupportedActorRef extends ActorRef with ScalaActorRef {
private[akka] final val uuid: akka.actor.Uuid = newUuid() private[akka] final val uuid: akka.actor.Uuid = newUuid()
def startsMonitoring(actorRef: ActorRef): ActorRef = actorRef def startsWatching(actorRef: ActorRef): ActorRef = actorRef
def stopsMonitoring(actorRef: ActorRef): ActorRef = actorRef def stopsWatching(actorRef: ActorRef): ActorRef = actorRef
def suspend(): Unit = () def suspend(): Unit = ()
@ -345,8 +345,8 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef {
private[akka] val uuid: Uuid = newUuid() private[akka] val uuid: Uuid = newUuid()
def name: String = uuid.toString def name: String = uuid.toString
def startsMonitoring(actorRef: ActorRef): ActorRef = actorRef def startsWatching(actorRef: ActorRef): ActorRef = actorRef
def stopsMonitoring(actorRef: ActorRef): ActorRef = actorRef def stopsWatching(actorRef: ActorRef): ActorRef = actorRef
def suspend(): Unit = () def suspend(): Unit = ()
def resume(): Unit = () def resume(): Unit = ()

View file

@ -59,8 +59,8 @@ case class Suspend() extends SystemMessage // sent to self from ActorCell.suspen
case class Resume() extends SystemMessage // sent to self from ActorCell.resume case class Resume() extends SystemMessage // sent to self from ActorCell.resume
case class Terminate() extends SystemMessage // sent to self from ActorCell.stop case class Terminate() extends SystemMessage // sent to self from ActorCell.stop
case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsMonitoring case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsWatching
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsMonitoring case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsWatching
final case class TaskInvocation(app: ActorSystem, function: () Unit, cleanup: () Unit) extends Runnable { final case class TaskInvocation(app: ActorSystem, function: () Unit, cleanup: () Unit) extends Runnable {
def run() { def run() {

View file

@ -293,8 +293,8 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
callback.done(false) callback.done(false)
} }
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported def startsWatching(actorRef: ActorRef): ActorRef = unsupported
def stopsMonitoring(actorRef: ActorRef): ActorRef = unsupported def stopsWatching(actorRef: ActorRef): ActorRef = unsupported
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
new KeptPromise[Any](Left(new UnsupportedOperationException("Ask/? is not supported for %s".format(getClass.getName)))) new KeptPromise[Any](Left(new UnsupportedOperationException("Ask/? is not supported for %s".format(getClass.getName))))

View file

@ -180,8 +180,8 @@ The messages that it prevents are all that extends 'LifeCycleMessage':
It also prevents the client from invoking any life-cycle and side-effecting methods, such as: It also prevents the client from invoking any life-cycle and side-effecting methods, such as:
* start * start
* stop * stop
* startsMonitoring * startsWatching
* stopsMonitoring * stopsWatching
* etc. * etc.
Using secure cookie for remote client authentication Using secure cookie for remote client authentication

View file

@ -182,8 +182,8 @@ The messages that it prevents are all that extends 'LifeCycleMessage':
It also prevents the client from invoking any life-cycle and side-effecting methods, such as: It also prevents the client from invoking any life-cycle and side-effecting methods, such as:
* start * start
* stop * stop
* startsMonitoring * startsWatching
* stopsMonitoring * stopsWatching
* etc. * etc.
Using secure cookie for remote client authentication Using secure cookie for remote client authentication

View file

@ -291,9 +291,9 @@ private[akka] case class RemoteActorRef private[akka] (
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = remote.app.provider.serialize(this) private def writeReplace(): AnyRef = remote.app.provider.serialize(this)
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement def startsWatching(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement
def stopsMonitoring(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement def stopsWatching(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement
protected[akka] def restart(cause: Throwable): Unit = () protected[akka] def restart(cause: Throwable): Unit = ()

View file

@ -155,7 +155,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
"stop when sent a poison pill" in { "stop when sent a poison pill" in {
EventFilter[ActorKilledException]() intercept { EventFilter[ActorKilledException]() intercept {
val a = TestActorRef(Props[WorkerActor]) val a = TestActorRef(Props[WorkerActor])
testActor startsMonitoring a testActor startsWatching a
a.!(PoisonPill)(testActor) a.!(PoisonPill)(testActor)
expectMsgPF(5 seconds) { expectMsgPF(5 seconds) {
case Terminated(`a`) true case Terminated(`a`) true