2011-10-10 11:12:34 +02:00
|
|
|
/**
|
2012-01-19 18:21:06 +01:00
|
|
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
2011-10-10 11:12:34 +02:00
|
|
|
*/
|
|
|
|
|
package akka.testkit
|
|
|
|
|
|
2011-11-16 19:34:37 +01:00
|
|
|
import org.scalatest.{ WordSpec, BeforeAndAfterAll, Tag }
|
2011-10-10 11:12:34 +02:00
|
|
|
import org.scalatest.matchers.MustMatchers
|
2012-05-15 16:01:32 +02:00
|
|
|
import akka.actor.ActorSystem
|
2011-10-10 15:45:55 +02:00
|
|
|
import akka.actor.{ Actor, ActorRef, Props }
|
2011-11-03 18:56:20 +01:00
|
|
|
import akka.event.{ Logging, LoggingAdapter }
|
2011-10-28 17:15:10 +02:00
|
|
|
import akka.util.duration._
|
2011-11-15 11:34:39 +01:00
|
|
|
import com.typesafe.config.Config
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
2011-12-05 15:18:22 +01:00
|
|
|
import akka.actor.PoisonPill
|
2011-12-05 18:52:32 +01:00
|
|
|
import akka.actor.DeadLetter
|
2011-12-11 00:40:52 +01:00
|
|
|
import java.util.concurrent.TimeoutException
|
2011-12-12 22:50:08 +01:00
|
|
|
import akka.dispatch.{ Await, MessageDispatcher }
|
2011-12-21 19:02:06 +01:00
|
|
|
import akka.dispatch.Dispatchers
|
2012-01-18 10:18:51 +01:00
|
|
|
import akka.pattern.ask
|
2011-10-10 11:12:34 +02:00
|
|
|
|
2011-11-16 19:34:37 +01:00
|
|
|
object TimingTest extends Tag("timing")
|
2012-02-19 21:18:16 +01:00
|
|
|
object LongRunningTest extends Tag("long-running")
|
2011-11-16 19:34:37 +01:00
|
|
|
|
2011-11-15 11:34:39 +01:00
|
|
|
object AkkaSpec {
|
2011-12-05 10:41:36 +01:00
|
|
|
val testConf: Config = ConfigFactory.parseString("""
|
2011-11-15 11:34:39 +01:00
|
|
|
akka {
|
|
|
|
|
event-handlers = ["akka.testkit.TestEventListener"]
|
|
|
|
|
loglevel = "WARNING"
|
2011-11-28 14:59:41 +01:00
|
|
|
stdout-loglevel = "WARNING"
|
2011-11-15 11:34:39 +01:00
|
|
|
actor {
|
|
|
|
|
default-dispatcher {
|
2012-02-01 10:10:13 +01:00
|
|
|
executor = "fork-join-executor"
|
|
|
|
|
fork-join-executor {
|
|
|
|
|
parallelism-min = 8
|
|
|
|
|
parallelism-factor = 2.0
|
|
|
|
|
parallelism-max = 8
|
2012-01-30 16:34:25 +01:00
|
|
|
}
|
2011-11-15 11:34:39 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2011-11-29 11:50:22 +01:00
|
|
|
""")
|
2011-11-15 11:34:39 +01:00
|
|
|
|
|
|
|
|
def mapToConfig(map: Map[String, Any]): Config = {
|
|
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
ConfigFactory.parseMap(map.asJava)
|
|
|
|
|
}
|
|
|
|
|
|
2012-05-24 13:56:50 +02:00
|
|
|
def getCallerName(clazz: Class[_]): String = {
|
2011-11-30 22:19:48 +01:00
|
|
|
val s = Thread.currentThread.getStackTrace map (_.getClassName) drop 1 dropWhile (_ matches ".*AkkaSpec.?$")
|
2012-05-24 13:56:50 +02:00
|
|
|
val reduced = s.lastIndexWhere(_ == clazz.getName) match {
|
|
|
|
|
case -1 ⇒ s
|
|
|
|
|
case z ⇒ s drop (z + 1)
|
|
|
|
|
}
|
|
|
|
|
reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
|
2011-11-30 22:19:48 +01:00
|
|
|
}
|
|
|
|
|
|
2011-11-15 11:34:39 +01:00
|
|
|
}
|
|
|
|
|
|
2011-11-30 22:19:48 +01:00
|
|
|
abstract class AkkaSpec(_system: ActorSystem)
|
2011-11-22 13:04:10 +01:00
|
|
|
extends TestKit(_system) with WordSpec with MustMatchers with BeforeAndAfterAll {
|
2011-10-20 20:45:02 +02:00
|
|
|
|
2012-05-24 13:56:50 +02:00
|
|
|
def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName(getClass), config.withFallback(AkkaSpec.testConf)))
|
2011-11-30 22:19:48 +01:00
|
|
|
|
2011-12-02 13:31:48 +01:00
|
|
|
def this(s: String) = this(ConfigFactory.parseString(s))
|
2011-11-30 22:19:48 +01:00
|
|
|
|
|
|
|
|
def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap))
|
|
|
|
|
|
2012-05-24 13:56:50 +02:00
|
|
|
def this() = this(ActorSystem(AkkaSpec.getCallerName(getClass), AkkaSpec.testConf))
|
2011-11-30 22:19:48 +01:00
|
|
|
|
2011-11-18 11:59:43 +01:00
|
|
|
val log: LoggingAdapter = Logging(system, this.getClass)
|
2011-10-27 12:23:01 +02:00
|
|
|
|
2011-10-20 20:45:02 +02:00
|
|
|
final override def beforeAll {
|
|
|
|
|
atStartup()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final override def afterAll {
|
2011-12-14 01:06:20 +01:00
|
|
|
system.shutdown()
|
2012-05-15 16:01:32 +02:00
|
|
|
try system.awaitTermination(5 seconds) catch {
|
2011-12-11 00:40:52 +01:00
|
|
|
case _: TimeoutException ⇒ system.log.warning("Failed to stop [{}] within 5 seconds", system.name)
|
2011-10-28 17:15:10 +02:00
|
|
|
}
|
2011-10-20 20:45:02 +02:00
|
|
|
atTermination()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
protected def atStartup() {}
|
|
|
|
|
|
|
|
|
|
protected def atTermination() {}
|
2011-10-10 11:12:34 +02:00
|
|
|
|
2011-12-21 19:02:06 +01:00
|
|
|
def spawn(dispatcherId: String = Dispatchers.DefaultDispatcherId)(body: ⇒ Unit) {
|
|
|
|
|
system.actorOf(Props(ctx ⇒ { case "go" ⇒ try body finally ctx.stop(ctx.self) }).withDispatcher(dispatcherId)) ! "go"
|
2011-10-11 16:05:48 +02:00
|
|
|
}
|
2011-10-28 17:15:10 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-03 18:56:20 +01:00
|
|
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
2011-10-28 17:15:10 +02:00
|
|
|
class AkkaSpecSpec extends WordSpec with MustMatchers {
|
2011-12-05 15:18:22 +01:00
|
|
|
|
2011-10-28 17:15:10 +02:00
|
|
|
"An AkkaSpec" must {
|
2011-12-05 15:18:22 +01:00
|
|
|
|
2012-04-23 15:40:21 +02:00
|
|
|
"warn about unhandled messages" in {
|
|
|
|
|
implicit val system = ActorSystem("AkkaSpec0", AkkaSpec.testConf)
|
|
|
|
|
try {
|
|
|
|
|
val a = system.actorOf(Props.empty)
|
|
|
|
|
EventFilter.warning(start = "unhandled message", occurrences = 1) intercept {
|
|
|
|
|
a ! 42
|
|
|
|
|
}
|
|
|
|
|
} finally {
|
|
|
|
|
system.shutdown()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2011-10-28 17:15:10 +02:00
|
|
|
"terminate all actors" in {
|
2011-12-05 18:52:32 +01:00
|
|
|
// verbose config just for demonstration purposes, please leave in in case of debugging
|
2011-11-15 11:34:39 +01:00
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
val conf = Map(
|
2011-11-12 22:37:12 +01:00
|
|
|
"akka.actor.debug.lifecycle" -> true, "akka.actor.debug.event-stream" -> true,
|
2011-11-15 11:34:39 +01:00
|
|
|
"akka.loglevel" -> "DEBUG", "akka.stdout-loglevel" -> "DEBUG")
|
2011-12-05 18:52:32 +01:00
|
|
|
val system = ActorSystem("AkkaSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(AkkaSpec.testConf))
|
2012-05-29 14:09:22 +02:00
|
|
|
val spec = new AkkaSpec(system) { val ref = Seq(testActor, system.actorOf(Props.empty, "name")) }
|
2012-05-03 21:14:47 +02:00
|
|
|
spec.ref foreach (_.isTerminated must not be true)
|
2011-12-14 01:06:20 +01:00
|
|
|
system.shutdown()
|
2012-05-03 21:14:47 +02:00
|
|
|
spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds)
|
2011-10-28 17:15:10 +02:00
|
|
|
}
|
2011-12-05 15:18:22 +01:00
|
|
|
|
|
|
|
|
"must stop correctly when sending PoisonPill to rootGuardian" in {
|
2011-12-05 18:52:32 +01:00
|
|
|
val system = ActorSystem("AkkaSpec2", AkkaSpec.testConf)
|
2011-12-05 15:18:22 +01:00
|
|
|
val spec = new AkkaSpec(system) {}
|
|
|
|
|
val latch = new TestLatch(1)(system)
|
|
|
|
|
system.registerOnTermination(latch.countDown())
|
|
|
|
|
|
|
|
|
|
system.actorFor("/") ! PoisonPill
|
|
|
|
|
|
2011-12-19 15:05:33 +01:00
|
|
|
Await.ready(latch, 2 seconds)
|
2011-12-05 15:18:22 +01:00
|
|
|
}
|
|
|
|
|
|
2011-12-05 18:52:32 +01:00
|
|
|
"must enqueue unread messages from testActor to deadLetters" in {
|
2011-12-05 22:46:34 +01:00
|
|
|
val system, otherSystem = ActorSystem("AkkaSpec3", AkkaSpec.testConf)
|
|
|
|
|
|
|
|
|
|
try {
|
|
|
|
|
var locker = Seq.empty[DeadLetter]
|
2012-02-10 16:02:37 +01:00
|
|
|
implicit val timeout = TestKitExtension(system).DefaultTimeout
|
2011-12-05 22:46:34 +01:00
|
|
|
implicit val davyJones = otherSystem.actorOf(Props(new Actor {
|
|
|
|
|
def receive = {
|
|
|
|
|
case m: DeadLetter ⇒ locker :+= m
|
2011-12-14 00:06:36 +01:00
|
|
|
case "Die!" ⇒ sender ! "finally gone"; context.stop(self)
|
2011-12-05 22:46:34 +01:00
|
|
|
}
|
|
|
|
|
}), "davyJones")
|
|
|
|
|
|
|
|
|
|
system.eventStream.subscribe(davyJones, classOf[DeadLetter])
|
|
|
|
|
|
|
|
|
|
val probe = new TestProbe(system)
|
|
|
|
|
probe.ref ! 42
|
|
|
|
|
/*
|
|
|
|
|
* this will ensure that the message is actually received, otherwise it
|
2011-12-20 21:08:27 +01:00
|
|
|
* may happen that the system.stop() suspends the testActor before it had
|
2011-12-05 22:46:34 +01:00
|
|
|
* a chance to put the message into its private queue
|
|
|
|
|
*/
|
|
|
|
|
probe.receiveWhile(1 second) {
|
|
|
|
|
case null ⇒
|
2011-12-05 18:52:32 +01:00
|
|
|
}
|
|
|
|
|
|
2011-12-05 22:46:34 +01:00
|
|
|
val latch = new TestLatch(1)(system)
|
|
|
|
|
system.registerOnTermination(latch.countDown())
|
2011-12-14 01:06:20 +01:00
|
|
|
system.shutdown()
|
2011-12-19 15:05:33 +01:00
|
|
|
Await.ready(latch, 2 seconds)
|
2011-12-12 22:50:08 +01:00
|
|
|
Await.result(davyJones ? "Die!", timeout.duration) must be === "finally gone"
|
2011-12-05 18:52:32 +01:00
|
|
|
|
2011-12-05 22:46:34 +01:00
|
|
|
// this will typically also contain log messages which were sent after the logger shutdown
|
|
|
|
|
locker must contain(DeadLetter(42, davyJones, probe.ref))
|
|
|
|
|
} finally {
|
2011-12-14 01:06:20 +01:00
|
|
|
system.shutdown()
|
|
|
|
|
otherSystem.shutdown()
|
2011-12-05 22:46:34 +01:00
|
|
|
}
|
2011-12-05 18:52:32 +01:00
|
|
|
}
|
|
|
|
|
|
2011-10-28 17:15:10 +02:00
|
|
|
}
|
2011-11-09 14:56:05 +01:00
|
|
|
}
|
|
|
|
|
|