+act - 15757 - Reworks implementation of ActorSystem shutdown

* deprecates awaitTermination, shutdown and isTerminated
  * introduces a terminate-method that returns a Future[Unit]
  * introduces a whenTerminated-method that returns a Future[Unit]
  * simplifies the implementation by removing blocking constructs
  * adds tests for terminate() and whenTerminated
This commit is contained in:
Viktor Klang 2014-08-25 15:49:28 +02:00
parent 3ca27a54ad
commit cd8e97c060
51 changed files with 183 additions and 169 deletions

View file

@ -195,7 +195,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
}
}
system2.shutdown()
system2.terminate()
Await.ready(latch, 5 seconds)
val expected = (for (i 1 to count) yield i).reverse
@ -213,23 +213,28 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
callbackWasRun = true
}
import system.dispatcher
system2.scheduler.scheduleOnce(200.millis.dilated) { system2.shutdown() }
system2.scheduler.scheduleOnce(200.millis.dilated) { system2.terminate() }
system2.awaitTermination(5 seconds)
Await.ready(system2.whenTerminated, 5 seconds)
callbackWasRun should be(true)
}
"return isTerminated status correctly" in {
val system = ActorSystem()
system.isTerminated should be(false)
system.shutdown()
val wt = system.whenTerminated
wt.isCompleted should be(false)
val f = system.terminate()
Await.ready(wt, 10 seconds)
Await.ready(f, 10 seconds)
system.awaitTermination(10 seconds)
system.isTerminated should be(true)
}
"throw RejectedExecutionException when shutdown" in {
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
system2.shutdown()
Await.ready(system2.terminate(), 10 seconds)
system2.awaitTermination(10 seconds)
intercept[RejectedExecutionException] {
@ -252,10 +257,10 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
"reliable deny creation of actors while shutting down" in {
val system = ActorSystem()
import system.dispatcher
system.scheduler.scheduleOnce(200 millis) { system.shutdown() }
system.scheduler.scheduleOnce(200 millis) { system.terminate() }
var failing = false
var created = Vector.empty[ActorRef]
while (!system.isTerminated) {
while (!system.whenTerminated.isCompleted) {
try {
val t = system.actorOf(Props[ActorSystemSpec.Terminater])
failing should not be true // because once failing => always failing (its due to shutdown)
@ -278,7 +283,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
implicit val system = ActorSystem("Stop", AkkaSpec.testConf)
EventFilter[ActorKilledException]() intercept {
system.actorSelection("/user") ! Kill
awaitCond(system.isTerminated)
Await.ready(system.whenTerminated, Duration.Inf)
}
}
@ -313,7 +318,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
}))
EventFilter[Exception]("hello") intercept {
a ! "die"
awaitCond(system.isTerminated)
Await.ready(system.whenTerminated, Duration.Inf)
}
}

View file

@ -111,7 +111,7 @@ object Chameneos {
val actor = system.actorOf(Props(new Mall(1000000, 4)))
Thread.sleep(10000)
println("Elapsed: " + (end - start))
system.shutdown()
system.terminate()
}
def main(args: Array[String]): Unit = run()

View file

@ -33,7 +33,7 @@ object Main {
val app = system.actorOf(Props(appClass), "app")
val terminator = system.actorOf(Props(classOf[Terminator], app), "app-terminator")
} catch {
case NonFatal(e) system.shutdown(); throw e
case NonFatal(e) system.terminate(); throw e
}
}
}
@ -43,7 +43,7 @@ object Main {
def receive = {
case Terminated(_)
log.info("application supervisor has terminated, shutting down")
context.system.shutdown()
context.system.terminate()
}
}

View file

@ -463,44 +463,52 @@ private[akka] class LocalActorRefProvider private[akka] (
override val deadLetters: InternalActorRef =
_deadLetters.getOrElse((p: ActorPath) new DeadLetterActorRef(this, p, eventStream)).apply(rootPath / "deadLetters")
private[this] final val terminationPromise: Promise[Unit] = Promise[Unit]()
def terminationFuture: Future[Unit] = terminationPromise.future
/*
* generate name for temporary actor refs
*/
private val tempNumber = new AtomicLong
private def tempName() = Helpers.base64(tempNumber.getAndIncrement())
private val tempNode = rootPath / "temp"
override def tempPath(): ActorPath = tempNode / tempName()
override def tempPath(): ActorPath = tempNode / Helpers.base64(tempNumber.getAndIncrement())
/**
* Top-level anchor for the supervision hierarchy of this actor system. Will
* receive only Supervise/ChildTerminated system messages or Failure message.
*/
private[akka] val theOneWhoWalksTheBubblesOfSpaceTime: InternalActorRef = new MinimalActorRef {
val stopped = new Switch(false)
@volatile
var causeOfTermination: Option[Throwable] = None
val causeOfTermination: Promise[Unit] = Promise[Unit]()
val path = rootPath / "bubble-walker"
def provider: ActorRefProvider = LocalActorRefProvider.this
override def stop(): Unit = stopped switchOn { terminationPromise.complete(causeOfTermination.map(Failure(_)).getOrElse(Success(()))) }
@deprecated("Use context.watch(actor) and receive Terminated(actor)", "2.2") override def isTerminated: Boolean = stopped.isOn
def isWalking = causeOfTermination.future.isCompleted == false
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = stopped.ifOff(message match {
case null throw new InvalidMessageException("Message is null")
case _ log.error(s"$this received unexpected message [$message]")
})
override def stop(): Unit = {
causeOfTermination.trySuccess(()) //Idempotent
terminationPromise.tryCompleteWith(causeOfTermination.future) // Signal termination downstream, idempotent
}
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
@deprecated("Use context.watch(actor) and receive Terminated(actor)", "2.2")
override def isTerminated: Boolean = !isWalking
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit =
if (isWalking)
message match {
case null throw new InvalidMessageException("Message is null")
case _ log.error(s"$this received unexpected message [$message]")
}
override def sendSystemMessage(message: SystemMessage): Unit = if (isWalking) {
message match {
case Failed(child, ex, _)
log.error(ex, s"guardian $child failed, shutting down!")
causeOfTermination = Some(ex)
causeOfTermination.tryFailure(ex) // FIXME: Is the order of this vs the next line important/right?
child.asInstanceOf[InternalActorRef].stop()
case Supervise(_, _) // TODO register child in some map to keep track of it and enable shutdown after all dead
case _: DeathWatchNotification stop()
@ -519,10 +527,6 @@ private[akka] class LocalActorRefProvider private[akka] (
@volatile
private var system: ActorSystemImpl = _
lazy val terminationPromise: Promise[Unit] = Promise[Unit]()
def terminationFuture: Future[Unit] = terminationPromise.future
@volatile
private var extraNames: Map[String, InternalActorRef] = Map()

View file

@ -6,6 +6,7 @@ package akka.actor
import java.io.Closeable
import java.util.concurrent.{ ConcurrentHashMap, ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
import java.util.concurrent.atomic.{ AtomicReference }
import java.util.concurrent.TimeUnit.MILLISECONDS
import com.typesafe.config.{ Config, ConfigFactory }
import akka.event._
@ -17,7 +18,7 @@ import akka.util._
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration.{ FiniteDuration, Duration }
import scala.concurrent.{ Await, Awaitable, CanAwait, Future, ExecutionContext, ExecutionContextExecutor }
import scala.concurrent.{ Await, Future, Promise, ExecutionContext, ExecutionContextExecutor }
import scala.util.{ Failure, Success, Try }
import scala.util.control.{ NonFatal, ControlThrowable }
@ -391,12 +392,14 @@ abstract class ActorSystem extends ActorRefFactory {
*
* @throws TimeoutException in case of timeout
*/
@deprecated("Use Await.result(whenTerminated, timeout) instead", "2.4")
def awaitTermination(timeout: Duration): Unit
/**
* Block current thread until the system has been shutdown. This will
* block until after all on termination callbacks have been run.
*/
@deprecated("Use Await.result(whenTerminated, Duration.Inf) instead", "2.4")
def awaitTermination(): Unit
/**
@ -405,6 +408,7 @@ abstract class ActorSystem extends ActorRefFactory {
* (below which the logging actors reside) and the execute all registered
* termination handlers (see [[ActorSystem.registerOnTermination]]).
*/
@deprecated("Use the terminate() method instead", "2.4")
def shutdown(): Unit
/**
@ -414,8 +418,23 @@ abstract class ActorSystem extends ActorRefFactory {
* returns `false`, the status is actually unknown, since it might have
* changed since you queried it.
*/
@deprecated("Use the whenTerminated method instead.", "2.4")
def isTerminated: Boolean
/**
* Terminates this actor system. This will stop the guardian actor, which in turn
* will recursively stop all its child actors, then the system guardian
* (below which the logging actors reside) and the execute all registered
* termination handlers (see [[ActorSystem.registerOnTermination]]).
*/
def terminate(): Future[Unit]
/**
* Returns a Future which will be completed after the ActorSystem has been terminated
* and termination hooks have been executed.
*/
def whenTerminated: Future[Unit]
/**
* Registers the provided extension and creates its payload, if this extension isn't already registered
* This method has putIfAbsent-semantics, this method can potentially block, waiting for the initialization
@ -533,7 +552,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
}
} else {
log.error(cause, "Uncaught fatal error from thread [{}] shutting down ActorSystem [{}]", thread.getName, name)
shutdown()
terminate()
}
}
}
@ -618,7 +637,9 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
override def reportFailure(t: Throwable): Unit = dispatcher reportFailure t
})
def terminationFuture: Future[Unit] = provider.terminationFuture
private[this] final val terminationCallbacks = new TerminationCallbacks(provider.terminationFuture)(dispatcher)
def whenTerminated: Future[Unit] = terminationCallbacks.terminationFuture
def lookupRoot: InternalActorRef = provider.rootGuardian
def guardian: LocalActorRef = provider.guardian
def systemGuardian: LocalActorRef = provider.systemGuardian
@ -638,29 +659,23 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
this
} catch {
case NonFatal(e)
try {
shutdown()
} catch { case NonFatal(_) Try(stopScheduler()) }
try terminate() catch { case NonFatal(_) Try(stopScheduler()) }
throw e
}
def start(): this.type = _start
private lazy val terminationCallbacks = {
implicit val d = dispatcher
val callbacks = new TerminationCallbacks
terminationFuture onComplete (_ callbacks.run)
callbacks
}
def registerOnTermination[T](code: T) { registerOnTermination(new Runnable { def run = code }) }
def registerOnTermination(code: Runnable) { terminationCallbacks.add(code) }
def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) }
def awaitTermination() = awaitTermination(Duration.Inf)
def isTerminated = terminationCallbacks.isTerminated
override def awaitTermination(timeout: Duration) { Await.ready(whenTerminated, timeout) }
override def awaitTermination() = awaitTermination(Duration.Inf)
override def isTerminated = whenTerminated.isCompleted
def shutdown(): Unit = {
override def shutdown(): Unit = terminate()
override def terminate(): Future[Unit] = {
if (!settings.LogDeadLettersDuringShutdown) logDeadLetterListener foreach stop
guardian.stop()
whenTerminated
}
@volatile var aborting = false
@ -673,7 +688,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
*/
def abort(): Unit = {
aborting = true
shutdown()
terminate()
}
//#create-scheduler
@ -798,44 +813,31 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
printNode(lookupRoot, "")
}
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
private val lock = new ReentrantGuard
private var callbacks: List[Runnable] = _ //non-volatile since guarded by the lock
lock withGuard { callbacks = Nil }
final class TerminationCallbacks[T](upStreamTerminated: Future[T])(implicit ec: ExecutionContext) {
private[this] final val done = Promise[T]()
private[this] final val ref = new AtomicReference(done)
private val latch = new CountDownLatch(1)
// onComplete never fires twice so safe to avoid nullcheck
upStreamTerminated onComplete { t ref.getAndSet(null).complete(t) }
final def add(callback: Runnable): Unit = {
latch.getCount match {
case 0 throw new RejectedExecutionException("Must be called prior to system shutdown.")
case _ lock withGuard {
if (latch.getCount == 0) throw new RejectedExecutionException("Must be called prior to system shutdown.")
else callbacks ::= callback
}
/**
* Adds a Runnable that will be executed on ActorSystem termination.
* Note that callbacks are executed in reverse order of insertion.
* @param r The callback to be executed on ActorSystem termination
* @throws RejectedExecutionException if called after ActorSystem has been terminated
*/
final def add(r: Runnable): Unit = {
@tailrec def addRec(r: Runnable, p: Promise[T]): Unit = ref.get match {
case null throw new RejectedExecutionException("ActorSystem already terminated.")
case some if ref.compareAndSet(some, p) some.completeWith(p.future.andThen { case _ r.run() })
case _ addRec(r, p)
}
addRec(r, Promise[T]())
}
final def run(): Unit = lock withGuard {
@tailrec def runNext(c: List[Runnable]): List[Runnable] = c match {
case Nil Nil
case callback :: rest
try callback.run() catch { case NonFatal(e) log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) }
runNext(rest)
}
try { callbacks = runNext(callbacks) } finally latch.countDown()
}
final def ready(atMost: Duration)(implicit permit: CanAwait): this.type = {
if (atMost.isFinite()) {
if (!latch.await(atMost.length, atMost.unit))
throw new TimeoutException("Await termination timed out after [%s]" format (atMost.toString))
} else latch.await()
this
}
final def result(atMost: Duration)(implicit permit: CanAwait): Unit = ready(atMost)
final def isTerminated: Boolean = latch.getCount == 0
/**
* Returns a Future which will be completed once all registered callbacks have been executed.
*/
def terminationFuture: Future[T] = done.future
}
}

View file

@ -4,6 +4,7 @@
package akka.cluster
import language.postfixOps
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
@ -237,7 +238,7 @@ abstract class ClusterDeathWatchSpec
enterBarrier("first-unavailable")
val timeout = remainingOrDefault
try system.awaitTermination(timeout) catch {
try Await.ready(system.whenTerminated, timeout) catch {
case _: TimeoutException
fail("Failed to stop [%s] within [%s] \n%s".format(system.name, timeout,
system.asInstanceOf[ActorSystemImpl].printTree))

View file

@ -53,10 +53,8 @@ abstract class RestartFirstSeedNodeSpec
override def afterAll(): Unit = {
runOn(seed1) {
if (seed1System.isTerminated)
shutdown(restartedSeed1System)
else
shutdown(seed1System)
shutdown(
if (seed1System.whenTerminated.isCompleted) restartedSeed1System else seed1System)
}
super.afterAll()
}

View file

@ -5,6 +5,7 @@ package akka.cluster
import language.postfixOps
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.ActorSystem
@ -159,7 +160,7 @@ abstract class UnreachableNodeJoinsAgainSpec
runOn(victim) {
val victimAddress = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
system.awaitTermination(10 seconds)
Await.ready(system.whenTerminated, 10 seconds)
// create new ActorSystem with same host:port
val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s"""
akka.remote.netty.tcp {

View file

@ -108,7 +108,7 @@ class MyActor extends Actor {
}
override def postStop() {
context.system.shutdown()
context.system.terminate()
}
//#business-logic-elided
}

View file

@ -26,7 +26,7 @@ class ClusterSingletonProxySpec extends WordSpecLike with Matchers with BeforeAn
}
}
override def afterAll() = testSystems.foreach(_.system.shutdown())
override def afterAll() = testSystems.foreach(_.system.terminate())
}
object ClusterSingletonProxySpec {

View file

@ -22,7 +22,6 @@ import com.typesafe.config.ConfigFactory;
import static akka.japi.Util.classTag;
import static akka.actor.SupervisorStrategy.resume;
import static akka.actor.SupervisorStrategy.restart;
import static akka.actor.SupervisorStrategy.stop;
import static akka.actor.SupervisorStrategy.escalate;
@ -76,12 +75,12 @@ public class FaultHandlingDocSample {
log.info("Current progress: {} %", progress.percent);
if (progress.percent >= 100.0) {
log.info("That's all, shutting down");
getContext().system().shutdown();
getContext().system().terminate();
}
} else if (msg == ReceiveTimeout.getInstance()) {
// No progress within 15 seconds, ServiceUnavailable
log.error("Shutting down due to unavailable service");
getContext().system().shutdown();
getContext().system().terminate();
} else {
unhandled(msg);
}

View file

@ -28,7 +28,7 @@ public class EchoServer {
watcher.tell(ackServer, ActorRef.noSender());
latch.await(10, TimeUnit.MINUTES);
} finally {
system.shutdown();
system.terminate();
}
}

View file

@ -678,7 +678,7 @@ actors does not respond (i.e. processing a message for extended periods of time
and therefore not receiving the stop command), this whole process will be
stuck.
Upon :meth:`ActorSystem.shutdown()`, the system guardian actors will be
Upon :meth:`ActorSystem.terminate()`, the system guardian actors will be
stopped, and the aforementioned process will ensure proper termination of the
whole system.

View file

@ -621,7 +621,7 @@ actors does not respond (i.e. processing a message for extended periods of time
and therefore not receiving the stop command), this whole process will be
stuck.
Upon :meth:`ActorSystem.shutdown()`, the system guardian actors will be
Upon :meth:`ActorSystem.terminate()`, the system guardian actors will be
stopped, and the aforementioned process will ensure proper termination of the
whole system.

View file

@ -672,7 +672,7 @@ actors does not respond (i.e. processing a message for extended periods of time
and therefore not receiving the stop command), this whole process will be
stuck.
Upon :meth:`ActorSystem.shutdown()`, the system guardian actors will be
Upon :meth:`ActorSystem.terminate()`, the system guardian actors will be
stopped, and the aforementioned process will ensure proper termination of the
whole system.

View file

@ -53,13 +53,13 @@ class Listener extends Actor with ActorLogging {
log.info("Current progress: {} %", percent)
if (percent >= 100.0) {
log.info("That's all, shutting down")
context.system.shutdown()
context.system.terminate()
}
case ReceiveTimeout =>
// No progress within 15 seconds, ServiceUnavailable
log.error("Shutting down due to unavailable service")
context.system.shutdown()
context.system.terminate()
}
}

View file

@ -22,7 +22,7 @@ object EchoServer extends App {
// make sure to stop the system so that the application stops
try run()
finally system.shutdown()
finally system.terminate()
def run(): Unit = {
import ActorDSL._

View file

@ -10,6 +10,9 @@ import java.net.InetSocketAddress
import akka.testkit.{ ImplicitSender, TestProbe, AkkaSpec }
import akka.util.ByteString
import scala.concurrent.Await
import scala.concurrent.duration.Duration
object PullReadingExample {
class Listener(monitor: ActorRef) extends Actor {
@ -77,7 +80,6 @@ class PullReadingSpec extends AkkaSpec with ImplicitSender {
client.send(connection, ResumeReading)
client.expectMsg(Received(ByteString("hello")))
system.shutdown()
system.awaitTermination
Await.ready(system.terminate(), Duration.Inf)
}
}

View file

@ -322,7 +322,7 @@ trait PersistenceDocSpec {
processor ! PersistentBatch(List(Persistent("a"), Persistent("b")))
//#batch-write
system.shutdown()
system.terminate()
}
new AnyRef {

View file

@ -284,7 +284,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
//#put-your-test-code-here
val probe = TestProbe()
probe.send(testActor, "hello")
try expectMsg("hello") catch { case NonFatal(e) => system.shutdown(); throw e }
try expectMsg("hello") catch { case NonFatal(e) => system.terminate(); throw e }
//#put-your-test-code-here
shutdown(system)

View file

@ -29,7 +29,7 @@ import scala.collection.JavaConverters._
* }
*
* def shutdown = {
* system.shutdown()
* system.terminate()
* }
* }
* }}}

View file

@ -31,9 +31,11 @@ trait Player { this: TestConductorExt ⇒
private var _client: ActorRef = _
private def client = _client match {
case null throw new IllegalStateException("TestConductor client not yet started")
case _ if system.isTerminated throw new IllegalStateException("TestConductor unavailable because system is shutdown; you need to startNewSystem() before this point")
case x x
case null
throw new IllegalStateException("TestConductor client not yet started")
case _ if system.whenTerminated.isCompleted
throw new IllegalStateException("TestConductor unavailable because system is terminated; you need to startNewSystem() before this point")
case x x
}
/**
@ -241,7 +243,7 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
// FIXME: Currently ignoring, needs support from Remoting
stay
case TerminateMsg(Left(false))
context.system.shutdown()
context.system.terminate()
stay
case TerminateMsg(Left(true))
context.system.asInstanceOf[ActorSystemImpl].abort()

View file

@ -82,7 +82,7 @@ abstract class ActorSystemActivator extends BundleActivator {
*/
def stop(context: BundleContext): Unit = {
registration foreach (_.unregister())
system foreach (_.shutdown())
system foreach (_.terminate())
}
/**

View file

@ -51,11 +51,11 @@ class PingPongActorSystemActivatorTest extends WordSpec with Matchers with PojoS
"stop the ActorSystem when bundle stops" in {
filterErrors() {
val system = serviceForType[ActorSystem]
system.isTerminated should be(false)
system.whenTerminated.isCompleted should be(false)
bundleForName(TEST_BUNDLE_NAME).stop()
system.awaitTermination()
system.isTerminated should be(true)
Await.ready(system.whenTerminated, Duration.Inf)
system.whenTerminated.isCompleted should be(true)
}
}
}

View file

@ -13,6 +13,8 @@ import akka.testkit._
import akka.persistence.AtLeastOnceDelivery.AtLeastOnceDeliverySnapshot
import akka.persistence.AtLeastOnceDelivery.UnconfirmedDelivery
import scala.concurrent.Await
import scala.concurrent.duration.Duration
object SerializerSpecConfigs {
val customSerializers = ConfigFactory.parseString(
@ -202,8 +204,7 @@ class MessageSerializerRemotingSpec extends AkkaSpec(remote.withFallback(customS
}
override def afterTermination() {
remoteSystem.shutdown()
remoteSystem.awaitTermination()
Await.ready(remoteSystem.terminate(), Duration.Inf)
}
"A message serializer" must {

View file

@ -5,6 +5,7 @@ package akka.remote
import language.postfixOps
import java.util.concurrent.TimeoutException
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.Actor
@ -76,8 +77,9 @@ abstract class RemoteDeploymentDeathWatchSpec
sleep()
// if the remote deployed actor is not removed the system will not shutdown
val timeout = remainingOrDefault
try system.awaitTermination(timeout) catch {
try Await.ready(system.whenTerminated, timeout) catch {
case _: TimeoutException
fail("Failed to stop [%s] within [%s] \n%s".format(system.name, timeout,
system.asInstanceOf[ActorSystemImpl].printTree))

View file

@ -4,6 +4,7 @@
package akka.remote
import language.postfixOps
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.Actor
@ -40,7 +41,7 @@ object RemoteNodeRestartDeathWatchMultiJvmSpec extends MultiNodeConfig {
def receive = {
case "shutdown"
sender() ! "shutdown-ack"
context.system.shutdown()
context.system.terminate()
case msg sender() ! msg
}
}
@ -100,7 +101,7 @@ abstract class RemoteNodeRestartDeathWatchSpec
enterBarrier("watch-established")
system.awaitTermination(30.seconds)
Await.ready(system.whenTerminated, 30.seconds)
val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s"""
akka.remote.netty.tcp {
@ -110,7 +111,7 @@ abstract class RemoteNodeRestartDeathWatchSpec
""").withFallback(system.settings.config))
freshSystem.actorOf(Props[Subject], "subject")
freshSystem.awaitTermination(30.seconds)
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
}

View file

@ -37,7 +37,7 @@ object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig {
class Subject extends Actor {
def receive = {
case "shutdown" context.system.shutdown()
case "shutdown" context.system.terminate()
case msg sender() ! msg
}
}
@ -133,7 +133,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec
enterBarrier("watch-established")
system.awaitTermination(30.seconds)
Await.ready(system.whenTerminated, 30.seconds)
val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s"""
akka.remote.netty.tcp {
@ -143,7 +143,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec
""").withFallback(system.settings.config))
freshSystem.actorOf(Props[Subject], "subject")
freshSystem.awaitTermination(30.seconds)
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
}

View file

@ -32,8 +32,8 @@ object RemoteQuarantinePiercingSpec extends MultiNodeConfig {
class Subject extends Actor {
def receive = {
case "shutdown" context.system.shutdown()
case "identify" sender() ! (AddressUidExtension(context.system).addressUid, self)
case "shutdown" context.system.terminate()
case "identify" sender() ! (AddressUidExtension(context.system).addressUid -> self)
}
}
@ -100,7 +100,7 @@ abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuaranti
enterBarrier("actor-identified")
system.awaitTermination(30.seconds)
Await.ready(system.whenTerminated, 30.seconds)
val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s"""
akka.remote.netty.tcp {
@ -110,7 +110,7 @@ abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuaranti
""").withFallback(system.settings.config))
freshSystem.actorOf(Props[Subject], "subject")
freshSystem.awaitTermination(30.seconds)
Await.ready(freshSystem.whenTerminated, 30.seconds)
}
}

View file

@ -10,6 +10,7 @@ import akka.actor.Props
import akka.remote.transport.ThrottlerTransportAdapter.Direction._
import com.typesafe.config.ConfigFactory
import akka.actor.ActorSystem
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.actor.ActorLogging
import akka.remote.testconductor.TestConductor
@ -124,7 +125,7 @@ abstract class RemoteReDeploymentMultiJvmSpec extends MultiNodeSpec(RemoteReDepl
var sys: ActorSystem = null
runOn(second) {
system.awaitTermination(30.seconds)
Await.ready(system.whenTerminated, 30.seconds)
expectNoMsg(sleepAfterKill)
sys = startNewSystem()
}

View file

@ -5,8 +5,6 @@
package docs.actor;
import akka.actor.*;
import akka.event.LoggingAdapter;
import akka.event.Logging;
import akka.japi.pf.ReceiveBuilder;
import akka.testkit.ErrorFilter;
import akka.testkit.EventFilter;
@ -65,9 +63,8 @@ public class ActorDocTest {
}
@AfterClass
public static void afterClass() {
system.shutdown();
system.awaitTermination(Duration.create("5 seconds"));
public static void afterClass() throws Exception {
Await.ready(system.terminate(), Duration.create("5 seconds"));
}
static
@ -316,7 +313,7 @@ public class ActorDocTest {
swapper.tell(Swap, ActorRef.noSender()); // logs Ho
swapper.tell(Swap, ActorRef.noSender()); // logs Hi
swapper.tell(Swap, ActorRef.noSender()); // logs Ho
system.shutdown();
system.terminate();
}
}
//#swapper

View file

@ -10,11 +10,11 @@ import akka.testkit.JavaTestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.PartialFunction;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import scala.concurrent.Await;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
public class InitializationDocTest {
@ -26,9 +26,8 @@ public class InitializationDocTest {
}
@AfterClass
public static void afterClass() {
system.shutdown();
system.awaitTermination(Duration.create("5 seconds"));
public static void afterClass() throws Exception {
Await.ready(system.terminate(), Duration.create("5 seconds"));
}
public static class MessageInitExample extends AbstractActor {

View file

@ -19,11 +19,8 @@ import akka.util.Timeout;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import scala.concurrent.duration.Duration;
import scala.PartialFunction;
import scala.runtime.BoxedUnit;
import static akka.japi.Util.classTag;
import static akka.actor.SupervisorStrategy.resume;
import static akka.actor.SupervisorStrategy.restart;
import static akka.actor.SupervisorStrategy.stop;
import static akka.actor.SupervisorStrategy.escalate;
@ -79,13 +76,13 @@ public class FaultHandlingDocSample {
log().info("Current progress: {} %", progress.percent);
if (progress.percent >= 100.0) {
log().info("That's all, shutting down");
context().system().shutdown();
context().system().terminate();
}
}).
matchEquals(ReceiveTimeout.getInstance(), x -> {
// No progress within 15 seconds, ServiceUnavailable
log().error("Shutting down due to unavailable service");
context().system().shutdown();
context().system().terminate();
}).build(), context()
));
}

View file

@ -38,6 +38,6 @@ public class HelloKernel implements Bootable {
}
public void shutdown() {
system.shutdown();
system.terminate();
}
}

View file

@ -29,6 +29,6 @@ class HelloKernel extends Bootable {
}
def shutdown = {
system.shutdown()
system.terminate()
}
}

View file

@ -21,7 +21,7 @@ public class Main2 {
receive(ReceiveBuilder.
match(Terminated.class, t -> {
log().info("{} has terminated, shutting down system", ref.path());
context().system().shutdown();
context().system().terminate();
}).build());
}
}

View file

@ -30,7 +30,7 @@ public class Main2 {
public void onReceive(Object msg) {
if (msg instanceof Terminated) {
log.info("{} has terminated, shutting down system", ref.path());
getContext().system().shutdown();
getContext().system().terminate();
} else {
unhandled(msg);
}

View file

@ -20,7 +20,7 @@ object Main2 {
def receive = {
case Terminated(_) =>
log.info("{} has terminated, shutting down system", ref.path)
context.system.shutdown()
context.system.terminate()
}
}

View file

@ -125,6 +125,6 @@ public class PersistentActorExample {
processor.tell("print", null);
Thread.sleep(1000);
system.shutdown();
system.terminate();
}
}

View file

@ -10,8 +10,6 @@ import akka.actor.Props;
import akka.japi.pf.ReceiveBuilder;
import akka.persistence.AbstractPersistentActor;
import akka.persistence.SnapshotOffer;
import scala.PartialFunction;
import scala.runtime.BoxedUnit;
import java.io.Serializable;
import java.util.ArrayList;
@ -88,6 +86,6 @@ public class SnapshotExample {
persistentActor.tell("print", null);
Thread.sleep(1000);
system.shutdown();
system.terminate();
}
}

View file

@ -127,6 +127,6 @@ public class PersistentActorExample {
persistentActor.tell("print", null);
Thread.sleep(1000);
system.shutdown();
system.terminate();
}
}

View file

@ -95,6 +95,6 @@ public class SnapshotExample {
persistentActor.tell("print", null);
Thread.sleep(1000);
system.shutdown();
system.terminate();
}
}

View file

@ -56,5 +56,5 @@ object PersistentActorExample extends App {
persistentActor ! "print"
Thread.sleep(1000)
system.shutdown()
system.terminate()
}

View file

@ -44,5 +44,5 @@ object SnapshotExample extends App {
persistentActor ! "print"
Thread.sleep(1000)
system.shutdown()
system.terminate()
}

View file

@ -17,7 +17,7 @@ class Receiver extends Actor {
def receive = {
case m: Echo => sender() ! m
case Shutdown => context.system.shutdown()
case Shutdown => context.system.terminate()
case _ =>
}
}

View file

@ -104,7 +104,7 @@ class Sender(path: String, totalMessages: Int, burstSize: Int, payloadSize: Int)
case Terminated(`actor`) =>
println("Receiver terminated")
context.system.shutdown()
context.system.terminate()
}

View file

@ -33,7 +33,6 @@ public class Main {
Integer result = Await.result(ask(calculatorService, task, new Timeout(duration)).mapTo(classTag(Integer.class)), duration);
System.out.println("Got result: " + result);
system.shutdown();
system.awaitTermination();
Await.ready(system.terminate(), Duration.Inf());
}
}

View file

@ -6,6 +6,7 @@ package akka.testkit
import language.postfixOps
import scala.annotation.{ varargs, tailrec }
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.reflect.ClassTag
import java.util.concurrent._
@ -698,10 +699,14 @@ trait TestKitBase {
*
* val test = system.actorOf(Props[SomeActor]
*
* within (1 second) {
* test ! SomeWork
* expectMsg(Result1) // bounded to 1 second
* expectMsg(Result2) // bounded to the remainder of the 1 second
* within (1 second) {
* test ! SomeWork
* expectMsg(Result1) // bounded to 1 second
* expectMsg(Result2) // bounded to the remainder of the 1 second
* }
*
* } finally {
* system.terminate()
* }
*
* } finally {
@ -768,8 +773,8 @@ object TestKit {
def shutdownActorSystem(actorSystem: ActorSystem,
duration: Duration = 10.seconds,
verifySystemShutdown: Boolean = false): Unit = {
actorSystem.shutdown()
try actorSystem.awaitTermination(duration) catch {
actorSystem.terminate()
try Await.ready(actorSystem.whenTerminated, duration) catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(actorSystem.name, duration,
actorSystem.asInstanceOf[ActorSystemImpl].printTree)

View file

@ -13,7 +13,7 @@ class DefaultTimeoutSpec
implicit lazy val system = ActorSystem("AkkaCustomSpec")
override def afterAll = system.shutdown
override def afterAll = system.terminate
"A spec with DefaultTimeout" should {
"use timeout from settings" in {

View file

@ -13,7 +13,7 @@ class ImplicitSenderSpec
implicit lazy val system = ActorSystem("AkkaCustomSpec")
override def afterAll = system.shutdown
override def afterAll = system.terminate
"An ImplicitSender" should {
"have testActor as its self" in {

View file

@ -457,7 +457,7 @@ object AkkaBuild extends Build {
|var remoteConfig = ConfigFactory.parseString("akka.remote.netty{port=0,use-dispatcher-for-io=akka.actor.default-dispatcher,execution-pool-size=0},akka.actor.provider=akka.remote.RemoteActorRefProvider").withFallback(config)
|var system: ActorSystem = null
|implicit def _system = system
|def startSystem(remoting: Boolean = false) { system = ActorSystem("repl", if(remoting) remoteConfig else config); println("dont forget to system.shutdown()!") }
|def startSystem(remoting: Boolean = false) { system = ActorSystem("repl", if(remoting) remoteConfig else config); println("dont forget to system.terminate()!") }
|implicit def ec = system.dispatcher
|implicit val timeout = Timeout(5 seconds)
|""".stripMargin,