Fixing FIXME to rename isShutdown to isTerminated

This commit is contained in:
Viktor Klang 2011-11-23 19:03:56 +01:00
parent 7d9a124b64
commit c56341b3a6
17 changed files with 45 additions and 46 deletions

View file

@ -80,11 +80,11 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach {
filterEvents(EventFilter[Exception]("Expected exception")) { filterEvents(EventFilter[Exception]("Expected exception")) {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0))))
val actor = (supervisor ? Props[CrashingActor]).as[ActorRef].get val actor = (supervisor ? Props[CrashingActor]).as[ActorRef].get
actor.isShutdown must be(false) actor.isTerminated must be(false)
actor ! "Die" actor ! "Die"
state.finished.await state.finished.await
1.second.dilated.sleep() 1.second.dilated.sleep()
actor.isShutdown must be(true) actor.isTerminated must be(true)
supervisor.stop() supervisor.stop()
} }
} }

View file

@ -364,7 +364,7 @@ class ActorRefSpec extends AkkaSpec {
ffive.get must be("five") ffive.get must be("five")
fnull.get must be("null") fnull.get must be("null")
awaitCond(ref.isShutdown, 2000 millis) awaitCond(ref.isTerminated, 2000 millis)
} }
"restart when Kill:ed" in { "restart when Kill:ed" in {

View file

@ -96,7 +96,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
terminal ! Kill terminal ! Kill
expectTerminationOf(terminal) expectTerminationOf(terminal)
terminal.isShutdown must be === true terminal.isTerminated must be === true
supervisor.stop() supervisor.stop()
} }
@ -123,7 +123,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
case FF(Failed(DeathPactException(`failed`))) if lastSender eq brother 2 case FF(Failed(DeathPactException(`failed`))) if lastSender eq brother 2
case Terminated(`brother`) 3 case Terminated(`brother`) 3
} }
testActor must not be 'shutdown testActor.isTerminated must not be true
result must be(Seq(1, 2, 3)) result must be(Seq(1, 2, 3))
} }
} }

View file

@ -129,7 +129,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
expectMsgPF() { expectMsgPF() {
case Logging.Debug(`name`, msg: String) if msg startsWith "received AutoReceiveMessage Envelope(PoisonPill" true case Logging.Debug(`name`, msg: String) if msg startsWith "received AutoReceiveMessage Envelope(PoisonPill" true
} }
awaitCond(actor.isShutdown, 100 millis) awaitCond(actor.isTerminated, 100 millis)
} }
} }

View file

@ -89,7 +89,7 @@ class RestartStrategySpec extends AkkaSpec {
(1 to 100) foreach { _ slave ! Crash } (1 to 100) foreach { _ slave ! Crash }
assert(countDownLatch.await(120, TimeUnit.SECONDS)) assert(countDownLatch.await(120, TimeUnit.SECONDS))
assert(!slave.isShutdown) assert(!slave.isTerminated)
} }
"ensure that slave restarts after number of crashes not within time range" in { "ensure that slave restarts after number of crashes not within time range" in {
@ -146,7 +146,7 @@ class RestartStrategySpec extends AkkaSpec {
assert(thirdRestartLatch.tryAwait(1, TimeUnit.SECONDS)) assert(thirdRestartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(!slave.isShutdown) assert(!slave.isTerminated)
} }
"ensure that slave is not restarted after max retries" in { "ensure that slave is not restarted after max retries" in {
@ -183,7 +183,7 @@ class RestartStrategySpec extends AkkaSpec {
// test restart and post restart ping // test restart and post restart ping
assert(restartLatch.tryAwait(10, TimeUnit.SECONDS)) assert(restartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(!slave.isShutdown) assert(!slave.isTerminated)
// now crash again... should not restart // now crash again... should not restart
slave ! Crash slave ! Crash
@ -197,7 +197,7 @@ class RestartStrategySpec extends AkkaSpec {
slave ! Crash slave ! Crash
assert(stopLatch.tryAwait(10, TimeUnit.SECONDS)) assert(stopLatch.tryAwait(10, TimeUnit.SECONDS))
sleep(500L) sleep(500L)
assert(slave.isShutdown) assert(slave.isTerminated)
} }
"ensure that slave is not restarted within time range" in { "ensure that slave is not restarted within time range" in {
@ -237,7 +237,7 @@ class RestartStrategySpec extends AkkaSpec {
// test restart and post restart ping // test restart and post restart ping
assert(restartLatch.tryAwait(10, TimeUnit.SECONDS)) assert(restartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(!slave.isShutdown) assert(!slave.isTerminated)
// now crash again... should not restart // now crash again... should not restart
slave ! Crash slave ! Crash
@ -253,7 +253,7 @@ class RestartStrategySpec extends AkkaSpec {
assert(maxNoOfRestartsLatch.tryAwait(10, TimeUnit.SECONDS)) assert(maxNoOfRestartsLatch.tryAwait(10, TimeUnit.SECONDS))
sleep(500L) sleep(500L)
assert(slave.isShutdown) assert(slave.isTerminated)
} }
} }
} }

View file

@ -358,7 +358,7 @@ abstract class ActorModelSpec extends AkkaSpec {
System.err.println("Buddies left: ") System.err.println("Buddies left: ")
buddies.toArray foreach { buddies.toArray foreach {
case cell: ActorCell case cell: ActorCell
System.err.println(" - " + cell.self.path + " " + cell.isShutdown + " " + 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()))
} }
System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages + " ") System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages + " ")
@ -448,7 +448,7 @@ class DispatcherModelSpec extends ActorModelSpec {
a.stop a.stop
b.stop b.stop
while (!a.isShutdown && !b.isShutdown) {} //Busy wait for termination while (!a.isTerminated && !b.isTerminated) {} //Busy wait for termination
assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1)
assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1)
@ -485,7 +485,7 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
a.stop a.stop
b.stop b.stop
while (!a.isShutdown && !b.isShutdown) {} //Busy wait for termination while (!a.isTerminated && !b.isTerminated) {} //Busy wait for termination
assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1)
assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1)

View file

@ -32,7 +32,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1))) val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(system, props, impl.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isTerminated must be(false)
} }
"send message to connection" in { "send message to connection" in {
@ -86,7 +86,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1))) val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(system, props, impl.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isTerminated must be(false)
} }
//In this test a bunch of actors are created and each actor has its own counter. //In this test a bunch of actors are created and each actor has its own counter.
@ -195,7 +195,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1))) val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(system, props, impl.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isTerminated must be(false)
} }
"deliver a broadcast message" in { "deliver a broadcast message" in {
@ -313,7 +313,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0)))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0))))
val actor = new RoutedActorRef(system, props, impl.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isTerminated must be(false)
} }
"deliver one-way messages in a round robin fashion" in { "deliver one-way messages in a round robin fashion" in {

View file

@ -118,7 +118,7 @@ private[akka] class ActorCell(
@inline @inline
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
final def isShutdown: Boolean = mailbox.isClosed final def isTerminated: Boolean = mailbox.isClosed
final def start(): Unit = { final def start(): Unit = {
mailbox = dispatcher.createMailbox(this) mailbox = dispatcher.createMailbox(this)
@ -153,7 +153,7 @@ private[akka] class ActorCell(
final def children: Iterable[ActorRef] = childrenRefs.values.view.map(_.child) final def children: Iterable[ActorRef] = childrenRefs.values.view.map(_.child)
final def getChild(name: String): Option[ActorRef] = final def getChild(name: String): Option[ActorRef] =
if (isShutdown) None else childrenRefs.get(name).map(_.child) if (isTerminated) None else childrenRefs.get(name).map(_.child)
final def tell(message: Any, sender: ActorRef): Unit = final def tell(message: Any, sender: ActorRef): Unit =
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)) dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender))

View file

@ -126,7 +126,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
/** /**
* Is the actor shut down? * Is the actor shut down?
*/ */
def isShutdown: Boolean def isTerminated: Boolean
/** /**
* Registers this actor to be a death monitor of the provided ActorRef * Registers this actor to be a death monitor of the provided ActorRef
@ -189,11 +189,10 @@ class LocalActorRef private[akka] (
actorCell.start() actorCell.start()
/** /**
* Is the actor shut down? * Is the actor terminated?
* If this method returns true, it will never return false again, but if it returns false, you cannot be sure if it's alive still (race condition) * If this method returns true, it will never return false again, but if it returns false, you cannot be sure if it's alive still (race condition)
*/ */
//FIXME TODO RENAME TO isTerminated override def isTerminated: Boolean = actorCell.isTerminated
def isShutdown: Boolean = actorCell.isShutdown
/** /**
* Suspends the actor so that it will not process messages until resumed. The * Suspends the actor so that it will not process messages until resumed. The
@ -242,7 +241,7 @@ class LocalActorRef private[akka] (
// @deprecated("This method does a spin-lock to block for the actor, which might never be there, do not use this", "2.0") // @deprecated("This method does a spin-lock to block for the actor, which might never be there, do not use this", "2.0")
protected[akka] def underlyingActorInstance: Actor = { protected[akka] def underlyingActorInstance: Actor = {
var instance = actorCell.actor var instance = actorCell.actor
while ((instance eq null) && !actorCell.isShutdown) { while ((instance eq null) && !actorCell.isTerminated) {
try { Thread.sleep(1) } catch { case i: InterruptedException } try { Thread.sleep(1) } catch { case i: InterruptedException }
instance = actorCell.actor instance = actorCell.actor
} }
@ -339,7 +338,7 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef {
def stop(): Unit = () def stop(): Unit = ()
def isShutdown = false def isTerminated = false
def !(message: Any)(implicit sender: ActorRef = null): Unit = () def !(message: Any)(implicit sender: ActorRef = null): Unit = ()
@ -373,7 +372,7 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) exte
def address: String = path.toString def address: String = path.toString
override def isShutdown(): Boolean = true override def isTerminated(): Boolean = true
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match { override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
case d: DeadLetter eventStream.publish(d) case d: DeadLetter eventStream.publish(d)
@ -420,9 +419,9 @@ abstract class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deat
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = override def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
new KeptPromise[Any](Left(new UnsupportedOperationException("Ask/? is not supported for %s".format(getClass.getName))))(dispatcher) new KeptPromise[Any](Left(new UnsupportedOperationException("Ask/? is not supported for %s".format(getClass.getName))))(dispatcher)
override def isShutdown = result.isCompleted || result.isExpired override def isTerminated = result.isCompleted || result.isExpired
override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped")) override def stop(): Unit = if (!isTerminated) result.completeWithException(new ActorKilledException("Stopped"))
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = provider.serialize(this) private def writeReplace(): AnyRef = provider.serialize(this)

View file

@ -184,7 +184,7 @@ class LocalActorRefProvider(
override def stop() = stopped switchOn { terminationFuture.complete(causeOfTermination.toLeft(())) } override def stop() = stopped switchOn { terminationFuture.complete(causeOfTermination.toLeft(())) }
override def isShutdown = stopped.isOn override def isTerminated = stopped.isOn
override def !(message: Any)(implicit sender: ActorRef = null): Unit = stopped.ifOff(message match { override def !(message: Any)(implicit sender: ActorRef = null): Unit = stopped.ifOff(message match {
case Failed(ex) causeOfTermination = Some(ex); sender.stop() case Failed(ex) causeOfTermination = Some(ex); sender.stop()

View file

@ -246,19 +246,19 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
val current = mappings get monitored val current = mappings get monitored
current match { current match {
case null case null
if (monitored.isShutdown) false if (monitored.isTerminated) false
else { else {
if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor) if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor)
else if (monitored.isShutdown) !dissociate(monitored, monitor) else true else if (monitored.isTerminated) !dissociate(monitored, monitor) else true
} }
case raw: Vector[_] case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]] val v = raw.asInstanceOf[Vector[ActorRef]]
if (monitored.isShutdown) false if (monitored.isTerminated) false
if (v.contains(monitor)) true if (v.contains(monitor)) true
else { else {
val added = v :+ monitor val added = v :+ monitor
if (!mappings.replace(monitored, v, added)) associate(monitored, monitor) if (!mappings.replace(monitored, v, added)) associate(monitored, monitor)
else if (monitored.isShutdown) !dissociate(monitored, monitor) else true else if (monitored.isTerminated) !dissociate(monitored, monitor) else true
} }
} }
} }

View file

@ -177,7 +177,7 @@ private[akka] class RoutedActorRef(system: ActorSystem, val routedProps: RoutedP
@volatile @volatile
private var running: Boolean = true private var running: Boolean = true
override def isShutdown: Boolean = !running override def isTerminated: Boolean = !running
override def stop() { override def stop() {
synchronized { synchronized {
@ -411,7 +411,7 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
private def scatterGather[S, G >: S](message: Any, timeout: Timeout): Future[G] = { private def scatterGather[S, G >: S](message: Any, timeout: Timeout): Future[G] = {
val responses = connectionManager.connections.iterable.flatMap { actor val responses = connectionManager.connections.iterable.flatMap { actor
try { try {
if (actor.isShutdown) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace if (actor.isTerminated) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace
Some(actor.?(message, timeout).asInstanceOf[Future[S]]) Some(actor.?(message, timeout).asInstanceOf[Future[S]])
} catch { } catch {
case e: Exception case e: Exception

View file

@ -44,7 +44,7 @@ trait ListenerManagement {
def hasListeners: Boolean = !listeners.isEmpty def hasListeners: Boolean = !listeners.isEmpty
/** /**
* Checks if a specific listener is registered. Pruned eventually when isShutdown==true in notify. * Checks if a specific listener is registered. Pruned eventually when isTerminated==true in notify.
*/ */
def hasListener(listener: ActorRef): Boolean = listeners.contains(listener) def hasListener(listener: ActorRef): Boolean = listeners.contains(listener)
@ -54,7 +54,7 @@ trait ListenerManagement {
val iterator = listeners.iterator val iterator = listeners.iterator
while (iterator.hasNext) { while (iterator.hasNext) {
val listener = iterator.next val listener = iterator.next
if (listener.isShutdown) iterator.remove() if (listener.isTerminated) iterator.remove()
else listener ! msg else listener ! msg
} }
} }
@ -67,7 +67,7 @@ trait ListenerManagement {
val iterator = listeners.iterator val iterator = listeners.iterator
while (iterator.hasNext) { while (iterator.hasNext) {
val listener = iterator.next val listener = iterator.next
if (listener.isShutdown) iterator.remove() if (listener.isTerminated) iterator.remove()
else f(listener) else f(listener)
} }
} }

View file

@ -268,7 +268,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
@volatile @volatile
private var running: Boolean = true private var running: Boolean = true
def isShutdown: Boolean = !running def isTerminated: Boolean = !running
def suspend(): Unit = () def suspend(): Unit = ()

View file

@ -281,7 +281,7 @@ private[akka] case class RemoteActorRef private[akka] (
def address = remoteAddress + path.toString def address = remoteAddress + path.toString
def isShutdown: Boolean = !running def isTerminated: Boolean = !running
protected[akka] def sendSystemMessage(message: SystemMessage): Unit = unsupported protected[akka] def sendSystemMessage(message: SystemMessage): Unit = unsupported

View file

@ -95,9 +95,9 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
val spec = new AkkaSpec(system) { val spec = new AkkaSpec(system) {
val ref = Seq(testActor, system.actorOf(Props.empty, "name")) val ref = Seq(testActor, system.actorOf(Props.empty, "name"))
} }
spec.ref foreach (_ must not be 'shutdown) spec.ref foreach (_.isTerminated must not be true)
system.stop() system.stop()
spec.awaitCond(spec.ref forall (_.isShutdown), 2 seconds) spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds)
} }
} }
} }

View file

@ -160,7 +160,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
expectMsgPF(5 seconds) { expectMsgPF(5 seconds) {
case Terminated(`a`) true case Terminated(`a`) true
} }
a must be('shutdown) a.isTerminated must be(true)
assertThread assertThread
} }
} }
@ -224,7 +224,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
"proxy apply for the underlying actor" in { "proxy apply for the underlying actor" in {
val ref = TestActorRef[WorkerActor] val ref = TestActorRef[WorkerActor]
ref("work") ref("work")
ref.isShutdown must be(true) ref.isTerminated must be(true)
} }
} }