Removed all dependencies to ScalaTest in the published artifacts. See #1802

This commit is contained in:
Björn Antonsson 2012-09-12 15:12:13 +02:00
parent 6c25954a2d
commit afe30e9038
15 changed files with 146 additions and 54 deletions

View file

@ -9,7 +9,7 @@ import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor.{ Address, ExtendedActorSystem }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.{STMultiNodeSpec, MultiNodeSpec}
import akka.testkit._
import scala.concurrent.util.duration._
import scala.concurrent.util.Duration
@ -47,7 +47,7 @@ object MultiNodeClusterSpec {
""")
}
trait MultiNodeClusterSpec extends Suite { self: MultiNodeSpec
trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeSpec
override def initialParticipants = roles.size

View file

@ -4,18 +4,23 @@
package akka.remote.testkit
import language.implicitConversions
import language.postfixOps
import java.net.InetSocketAddress
import com.typesafe.config.{ ConfigObject, ConfigFactory, Config }
import akka.actor.{ RootActorPath, ActorPath, ActorSystem, ExtendedActorSystem }
import akka.actor._
import akka.util.Timeout
import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName }
import akka.remote.RemoteActorRefProvider
import akka.testkit.AkkaSpec
import akka.testkit.TestKit
import scala.concurrent.{ Await, Awaitable }
import scala.util.control.NonFatal
import scala.concurrent.util.Duration
import scala.concurrent.util.duration._
import java.util.concurrent.TimeoutException
import akka.remote.testconductor.RoleName
import akka.actor.RootActorPath
import akka.event.{ Logging, LoggingAdapter }
/**
* Configure the role names and participants of the test, including configuration settings.
@ -82,7 +87,7 @@ abstract class MultiNodeConfig {
}
private[testkit] def config: Config = {
val configs = (_nodeConf get myself).toList ::: _commonConf.toList ::: MultiNodeSpec.nodeConfig :: AkkaSpec.testConf :: Nil
val configs = (_nodeConf get myself).toList ::: _commonConf.toList ::: MultiNodeSpec.nodeConfig :: MultiNodeSpec.baseConfig :: Nil
configs reduce (_ withFallback _)
}
@ -119,12 +124,43 @@ object MultiNodeSpec {
require(selfIndex >= 0 && selfIndex < nodeNames.size, "selfIndex out of bounds: " + selfIndex)
val nodeConfig = AkkaSpec.mapToConfig(Map(
private[testkit] val nodeConfig = mapToConfig(Map(
"akka.actor.provider" -> "akka.remote.RemoteActorRefProvider",
"akka.remote.transport" -> "akka.remote.testconductor.TestConductorTransport",
"akka.remote.netty.hostname" -> nodeNames(selfIndex),
"akka.remote.netty.port" -> 0))
private[testkit] val baseConfig: Config = ConfigFactory.parseString("""
akka {
event-handlers = ["akka.testkit.TestEventListener"]
loglevel = "WARNING"
stdout-loglevel = "WARNING"
actor {
default-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = 8
parallelism-factor = 2.0
parallelism-max = 8
}
}
}
}
""")
private def mapToConfig(map: Map[String, Any]): Config = {
import scala.collection.JavaConverters._
ConfigFactory.parseMap(map.asJava)
}
private def getCallerName(clazz: Class[_]): String = {
val s = Thread.currentThread.getStackTrace map (_.getClassName) drop 1 dropWhile (_ matches ".*MultiNodeSpec.?$")
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]", "_")
}
}
/**
@ -136,18 +172,53 @@ object MultiNodeSpec {
* val is fine.
*/
abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: Seq[RoleName], deployments: RoleName Seq[String])
extends AkkaSpec(_system) {
extends TestKit(_system) with MultiNodeSpecCallbacks {
import MultiNodeSpec._
def this(config: MultiNodeConfig) =
this(config.myself, ActorSystem(AkkaSpec.getCallerName(classOf[MultiNodeSpec]), ConfigFactory.load(config.config)),
this(config.myself, ActorSystem(MultiNodeSpec.getCallerName(classOf[MultiNodeSpec]), ConfigFactory.load(config.config)),
config.roles, config.deployments)
val log: LoggingAdapter = Logging(system, this.getClass)
final override def multiNodeSpecBeforeAll {
atStartup()
}
final override def multiNodeSpecAfterAll {
// wait for all nodes to remove themselves before we shut the conductor down
if (selfIndex == 0) {
testConductor.removeNode(myself)
within(testConductor.Settings.BarrierTimeout.duration) {
awaitCond {
testConductor.getNodes.await.filterNot(_ == myself).isEmpty
}
}
}
system.shutdown()
try system.awaitTermination(5 seconds) catch {
case _: TimeoutException
system.log.warning("Failed to stop [{}] within 5 seconds", system.name)
println(system.asInstanceOf[ActorSystemImpl].printTree)
}
atTermination()
}
/*
* Test Class Interface
*/
/**
* Override this method to do something when the whole test is starting up.
*/
protected def atStartup(): Unit = {}
/**
* Override this method to do something when the whole test is terminating.
*/
protected def atTermination(): Unit = {}
/**
* All registered roles
*/
@ -267,16 +338,29 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
log.info("Role [{}] started with address [{}]", myself.name,
system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.address)
// wait for all nodes to remove themselves before we shut the conductor down
final override def beforeShutdown() = {
if (selfIndex == 0) {
testConductor.removeNode(myself)
within(testConductor.Settings.BarrierTimeout.duration) {
awaitCond {
testConductor.getNodes.await.filterNot(_ == myself).isEmpty
}
}
}
}
}
/**
* Use this to hook MultiNodeSpec into your test framework lifecycle, either by having your test extend MultiNodeSpec
* and override these methods or by creating a trait and then mixing that trait with your test together with MultiNodeSpec.
*
* Example trait for MultiNodeSpec with ScalaTest
*
* {{{
* trait STMultiNodeSpec extends MultiNodeSpecCallbacks with WordSpec with MustMatchers with BeforeAndAfterAll {
* override def beforeAll() = multiNodeSpecBeforeAll()
* override def afterAll() = multiNodeSpecAfterAll()
* }
* }}}
*/
trait MultiNodeSpecCallbacks {
/**
* Call this before the start of the test run. NOT before every test case.
*/
def multiNodeSpecBeforeAll(): Unit
/**
* Call this after the all test cases have run. NOT after every test case.
*/
def multiNodeSpecAfterAll(): Unit
}

View file

@ -7,8 +7,7 @@ import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import testkit.{STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec}
import akka.testkit._
object LookupRemoteActorMultiJvmSpec extends MultiNodeConfig {
@ -30,7 +29,7 @@ class LookupRemoteActorMultiJvmNode1 extends LookupRemoteActorSpec
class LookupRemoteActorMultiJvmNode2 extends LookupRemoteActorSpec
class LookupRemoteActorSpec extends MultiNodeSpec(LookupRemoteActorMultiJvmSpec)
with ImplicitSender with DefaultTimeout {
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
import LookupRemoteActorMultiJvmSpec._
def initialParticipants = 2

View file

@ -9,8 +9,7 @@ import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import testkit.{STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec}
import akka.testkit._
object NewRemoteActorMultiJvmSpec extends MultiNodeConfig {
@ -35,7 +34,7 @@ class NewRemoteActorMultiJvmNode1 extends NewRemoteActorSpec
class NewRemoteActorMultiJvmNode2 extends NewRemoteActorSpec
class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
with ImplicitSender with DefaultTimeout {
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
import NewRemoteActorMultiJvmSpec._
def initialParticipants = 2

View file

@ -12,8 +12,7 @@ import akka.actor.PoisonPill
import akka.actor.Address
import scala.concurrent.Await
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.{STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec}
import akka.routing.Broadcast
import akka.routing.RandomRouter
import akka.routing.RoutedActorRef
@ -48,7 +47,7 @@ class RandomRoutedRemoteActorMultiJvmNode3 extends RandomRoutedRemoteActorSpec
class RandomRoutedRemoteActorMultiJvmNode4 extends RandomRoutedRemoteActorSpec
class RandomRoutedRemoteActorSpec extends MultiNodeSpec(RandomRoutedRemoteActorMultiJvmSpec)
with ImplicitSender with DefaultTimeout {
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
import RandomRoutedRemoteActorMultiJvmSpec._
def initialParticipants = 4

View file

@ -12,8 +12,7 @@ import akka.actor.PoisonPill
import akka.actor.Address
import scala.concurrent.Await
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.{STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec}
import akka.routing.Broadcast
import akka.routing.RoundRobinRouter
import akka.routing.RoutedActorRef
@ -60,7 +59,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode3 extends RoundRobinRoutedRemoteAct
class RoundRobinRoutedRemoteActorMultiJvmNode4 extends RoundRobinRoutedRemoteActorSpec
class RoundRobinRoutedRemoteActorSpec extends MultiNodeSpec(RoundRobinRoutedRemoteActorMultiJvmSpec)
with ImplicitSender with DefaultTimeout {
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
import RoundRobinRoutedRemoteActorMultiJvmSpec._
def initialParticipants = 4

View file

@ -10,8 +10,7 @@ import akka.actor.ActorRef
import akka.actor.Props
import scala.concurrent.Await
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.{STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec}
import akka.routing.Broadcast
import akka.routing.ScatterGatherFirstCompletedRouter
import akka.routing.RoutedActorRef
@ -48,7 +47,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode3 extends ScatterGatherRoutedRem
class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends ScatterGatherRoutedRemoteActorSpec
class ScatterGatherRoutedRemoteActorSpec extends MultiNodeSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec)
with ImplicitSender with DefaultTimeout {
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
import ScatterGatherRoutedRemoteActorMultiJvmSpec._
def initialParticipants = 4

View file

@ -16,8 +16,7 @@ import akka.testkit.ImplicitSender
import akka.testkit.LongRunningTest
import java.net.InetSocketAddress
import java.net.InetAddress
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.{STMultiNodeSpec, MultiNodeSpec, MultiNodeConfig}
object TestConductorMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false))
@ -29,7 +28,7 @@ object TestConductorMultiJvmSpec extends MultiNodeConfig {
class TestConductorMultiJvmNode1 extends TestConductorSpec
class TestConductorMultiJvmNode2 extends TestConductorSpec
class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with ImplicitSender {
class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with STMultiNodeSpec with ImplicitSender {
import TestConductorMultiJvmSpec._

View file

@ -19,7 +19,7 @@ class MultiNodeSpecSpecMultiJvmNode2 extends MultiNodeSpecSpec
class MultiNodeSpecSpecMultiJvmNode3 extends MultiNodeSpecSpec
class MultiNodeSpecSpecMultiJvmNode4 extends MultiNodeSpecSpec
class MultiNodeSpecSpec extends MultiNodeSpec(MultiNodeSpecMultiJvmSpec) {
class MultiNodeSpecSpec extends MultiNodeSpec(MultiNodeSpecMultiJvmSpec) with STMultiNodeSpec {
import MultiNodeSpecMultiJvmSpec._

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote.testkit
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
import org.scalatest.matchers.MustMatchers
/**
* Hooks up MultiNodeSpec with ScalaTest
*/
trait STMultiNodeSpec extends MultiNodeSpecCallbacks with WordSpec with MustMatchers with BeforeAndAfterAll {
override def beforeAll() = multiNodeSpecBeforeAll()
override def afterAll() = multiNodeSpecAfterAll()
}

View file

@ -16,7 +16,7 @@ import java.util.concurrent.TimeoutException
import akka.dispatch.Dispatchers
import akka.pattern.ask
private[akka] object AkkaSpec {
object AkkaSpec {
val testConf: Config = ConfigFactory.parseString("""
akka {
event-handlers = ["akka.testkit.TestEventListener"]
@ -51,7 +51,7 @@ private[akka] object AkkaSpec {
}
private[akka] abstract class AkkaSpec(_system: ActorSystem)
abstract class AkkaSpec(_system: ActorSystem)
extends TestKit(_system) with WordSpec with MustMatchers with BeforeAndAfterAll {
def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName(getClass),
@ -70,7 +70,6 @@ private[akka] abstract class AkkaSpec(_system: ActorSystem)
}
final override def afterAll {
beforeShutdown()
system.shutdown()
try system.awaitTermination(5 seconds) catch {
case _: TimeoutException
@ -82,8 +81,6 @@ private[akka] abstract class AkkaSpec(_system: ActorSystem)
protected def atStartup() {}
protected def beforeShutdown() {}
protected def atTermination() {}
def spawn(dispatcherId: String = Dispatchers.DefaultDispatcherId)(body: Unit): Unit =

View file

@ -3,6 +3,9 @@
*/
package akka.testkit
import language.reflectiveCalls
import language.postfixOps
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.actor._

View file

@ -131,7 +131,7 @@ object AkkaBuild extends Build {
lazy val remoteTests = Project(
id = "akka-remote-tests-experimental",
base = file("akka-remote-tests"),
dependencies = Seq(remote, actorTests % "test->test", testkit % "compile;test->test"),
dependencies = Seq(remote, actorTests % "test->test", testkit),
settings = defaultSettings ++ multiJvmSettings ++ Seq(
libraryDependencies ++= Dependencies.remoteTests,
// disable parallel tests
@ -148,7 +148,7 @@ object AkkaBuild extends Build {
lazy val cluster = Project(
id = "akka-cluster-experimental",
base = file("akka-cluster"),
dependencies = Seq(remote, remoteTests % "test->test;multi-jvm->multi-jvm", testkit % "test->test"),
dependencies = Seq(remote, remoteTests % "test->test" , testkit % "test->test"),
settings = defaultSettings ++ multiJvmSettings ++ OSGi.cluster ++ Seq(
libraryDependencies ++= Dependencies.cluster,
// disable parallel tests
@ -549,13 +549,13 @@ object Dependencies {
val actor = Seq(config)
val testkit = Seq(Compile.scalatest, Compile.junit, Test.junit, Test.junit)
val testkit = Seq(Test.junit, Test.scalatest)
val actorTests = Seq(Test.junit, Test.scalatest, Test.commonsMath, Test.mockito, Test.scalacheck, protobuf)
val remote = Seq(netty, protobuf, uncommonsMath, Test.junit, Test.scalatest)
val remoteTests = Seq(Compile.scalatest, Test.junit, Test.scalatest)
val remoteTests = Seq(Test.junit, Test.scalatest)
val cluster = Seq(Test.junit, Test.scalatest)
@ -615,11 +615,6 @@ object Dependency {
val junitIntf = "com.novocode" % "junit-interface" % "0.8" % "test" // MIT
}
object Compile {
val junit = "junit" % "junit" % "4.10" % "compile" // Common Public License 1.0
val scalatest = "org.scalatest" % "scalatest" % "1.9-2.10.0-M7-B1" % "compile" cross CrossVersion.full // ApacheV2
}
// Camel Sample
object CamelSample {
val camelJetty = "org.apache.camel" % "camel-jetty" % "2.10.0" // ApacheV2