Making sure that OSGi docs don't break the build and making sure that the osgi artifacts are bundled in the project

This commit is contained in:
Viktor Klang 2012-07-01 13:31:11 +02:00
parent 34b05e554b
commit 3797b72c45
5 changed files with 53 additions and 76 deletions

View file

@ -1,18 +1,24 @@
import akka.actor.{ Props, ActorSystem } package docs.osgi
import akka.osgi.ActorSystemActivator
import org.apache.servicemix.examples.akka.Listener case object SomeMessage
import org.apache.servicemix.examples.akka.Master
class SomeActor extends akka.actor.Actor {
def receive = { case SomeMessage }
}
//#Activator //#Activator
class Activator extends ActorSystemActivator("PiSystem") { import akka.actor.{ Props, ActorSystem }
import org.osgi.framework.BundleContext
import akka.osgi.ActorSystemActivator
class Activator extends ActorSystemActivator {
def configure(context: BundleContext, system: ActorSystem) { def configure(context: BundleContext, system: ActorSystem) {
// optionally register the ActorSystem in the OSGi Service Registry // optionally register the ActorSystem in the OSGi Service Registry
registerService(context, system) registerService(context, system)
val listener = system.actorOf(Props[Listener], name = "listener") val someActor = system.actorOf(Props[SomeActor], name = "someName")
val master = system.actorOf(Props(new Master(4, 10000, 10000, listener)), name = "master") someActor ! SomeMessage
master ! Calculate
} }
} }

View file

@ -35,18 +35,17 @@ class SimpleNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT
"simple.xml" must { "simple.xml" must {
"set up ActorSystem when bundle starts" in { "set up ActorSystem when bundle starts" in {
val system = serviceForType[ActorSystem] serviceForType[ActorSystem] must not be (null)
assert(system != null)
} }
"stop the ActorSystem when bundle stops" in { "stop the ActorSystem when bundle stops" in {
val system = serviceForType[ActorSystem] val system = serviceForType[ActorSystem]
assert(!system.isTerminated) system.isTerminated must be(false)
bundleForName(TEST_BUNDLE_NAME).stop() bundleForName(TEST_BUNDLE_NAME).stop()
system.awaitTermination() system.awaitTermination()
assert(system.isTerminated) system.isTerminated must be(true)
} }
} }
@ -63,19 +62,19 @@ class ConfigNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT
"config.xml" must { "config.xml" must {
"set up ActorSystem when bundle starts" in { "set up ActorSystem when bundle starts" in {
val system = serviceForType[ActorSystem] val system = serviceForType[ActorSystem]
assert(system != null) system must not be (null)
assert(system.settings.config.getString("some.config.key") == "value") system.settings.config.getString("some.config.key") must be("value")
} }
"stop the ActorSystem when bundle stops" in { "stop the ActorSystem when bundle stops" in {
val system = serviceForType[ActorSystem] val system = serviceForType[ActorSystem]
assert(!system.isTerminated) system.isTerminated must be(false)
bundleForName(TEST_BUNDLE_NAME).stop() bundleForName(TEST_BUNDLE_NAME).stop()
system.awaitTermination() system.awaitTermination()
assert(system.isTerminated) system.isTerminated must be(true)
} }
} }
@ -93,8 +92,8 @@ class DependencyInjectionNamespaceHandlerTest extends WordSpec with MustMatchers
"set up bean containing ActorSystem" in { "set up bean containing ActorSystem" in {
val bean = serviceForType[ActorSystemAwareBean] val bean = serviceForType[ActorSystemAwareBean]
assert(bean != null) bean must not be (null)
assert(bean.system != null) bean.system must not be (null)
} }
} }

View file

@ -37,19 +37,17 @@ class PingPongActorSystemActivatorTest extends WordSpec with MustMatchers with P
val actor = system.actorFor("/user/pong") val actor = system.actorFor("/user/pong")
implicit val timeout = Timeout(5 seconds) implicit val timeout = Timeout(5 seconds)
val future = actor ? Ping Await.result(actor ? Ping, timeout.duration) must be(Pong)
val result = Await.result(future, timeout.duration)
assert(result != null)
} }
"stop the ActorSystem when bundle stops" in { "stop the ActorSystem when bundle stops" in {
val system = serviceForType[ActorSystem] val system = serviceForType[ActorSystem]
assert(!system.isTerminated) system.isTerminated must be(false)
bundleForName(TEST_BUNDLE_NAME).stop() bundleForName(TEST_BUNDLE_NAME).stop()
system.awaitTermination() system.awaitTermination()
assert(system.isTerminated) system.isTerminated must be(true)
} }
} }
@ -59,15 +57,12 @@ class RuntimeNameActorSystemActivatorTest extends WordSpec with MustMatchers wit
import ActorSystemActivatorTest._ import ActorSystemActivatorTest._
val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator])))
bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator])))
"RuntimeNameActorSystemActivator" must { "RuntimeNameActorSystemActivator" must {
"register an ActorSystem and add the bundle id to the system name" in { "register an ActorSystem and add the bundle id to the system name" in {
val system = serviceForType[ActorSystem] serviceForType[ActorSystem].name must equal(TestActivators.ACTOR_SYSTEM_NAME_PATTERN.format(bundleForName(TEST_BUNDLE_NAME).getBundleId))
val bundle = bundleForName(TEST_BUNDLE_NAME)
system.name must equal(TestActivators.ACTOR_SYSTEM_NAME_PATTERN.format(bundle.getBundleId))
} }
} }

View file

@ -36,10 +36,7 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll {
bundles.addAll(testBundles) bundles.addAll(testBundles)
config.put(PojoServiceRegistryFactory.BUNDLE_DESCRIPTORS, bundles) config.put(PojoServiceRegistryFactory.BUNDLE_DESCRIPTORS, bundles)
val loader: ServiceLoader[PojoServiceRegistryFactory] = ServiceLoader.load(classOf[PojoServiceRegistryFactory]) ServiceLoader.load(classOf[PojoServiceRegistryFactory]).iterator.next.newPojoServiceRegistry(config).getBundleContext
val registry = loader.iterator.next.newPojoServiceRegistry(config)
registry.getBundleContext
} }
// Ensure bundles get stopped at the end of the test to release resources and stop threads // Ensure bundles get stopped at the end of the test to release resources and stop threads
@ -48,25 +45,21 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll {
/** /**
* Convenience method to find a bundle by symbolic name * Convenience method to find a bundle by symbolic name
*/ */
def bundleForName(name: String) = context.getBundles.find(_.getSymbolicName == name) match { def bundleForName(name: String) =
case Some(bundle) bundle context.getBundles.find(_.getSymbolicName == name).getOrElse(fail("Unable to find bundle with symbolic name %s".format(name)))
case None fail("Unable to find bundle with symbolic name %s".format(name))
}
/** /**
* Convenience method to find a service by interface. If the service is not already available in the OSGi Service * Convenience method to find a service by interface. If the service is not already available in the OSGi Service
* Registry, this method will wait for a few seconds for the service to appear. * Registry, this method will wait for a few seconds for the service to appear.
*/ */
def serviceForType[T](implicit manifest: Manifest[T]): T = { def serviceForType[T](implicit manifest: Manifest[T]): T =
val reference = awaitReference(manifest.erasure) context.getService(awaitReference(manifest.erasure)).asInstanceOf[T]
context.getService(reference).asInstanceOf[T]
}
def awaitReference(serviceType: Class[_]): ServiceReference = awaitReference(serviceType, START_WAIT_TIME) def awaitReference(serviceType: Class[_]): ServiceReference = awaitReference(serviceType, START_WAIT_TIME)
def awaitReference(serviceType: Class[_], wait: Long): ServiceReference = { def awaitReference(serviceType: Class[_], wait: Long): ServiceReference = {
val option = Option(context.getServiceReference(serviceType.getName)) val option = Option(context.getServiceReference(serviceType.getName))
Thread.sleep(wait) Thread.sleep(wait) //FIXME No sleep please
option match { option match {
case Some(reference) reference case Some(reference) reference
case None if (wait > MAX_WAIT_TIME) fail("Gave up waiting for service of type %s".format(serviceType)) case None if (wait > MAX_WAIT_TIME) fail("Gave up waiting for service of type %s".format(serviceType))
@ -78,12 +71,10 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll {
} }
object PojoSRTestSupport { object PojoSRTestSupport {
/** /**
* Convenience method to define additional test bundles * Convenience method to define additional test bundles
*/ */
def bundle(name: String) = new BundleDescriptorBuilder(name) def bundle(name: String) = new BundleDescriptorBuilder(name)
} }
/** /**
@ -98,22 +89,24 @@ class BundleDescriptorBuilder(name: String) {
/** /**
* Add a Blueprint XML file to our test bundle * Add a Blueprint XML file to our test bundle
*/ */
def withBlueprintFile(name: String, contents: URL): BundleDescriptorBuilder = def withBlueprintFile(name: String, contents: URL): BundleDescriptorBuilder = {
returnBuilder(tinybundle.add("OSGI-INF/blueprint/%s".format(name), contents)) tinybundle.add("OSGI-INF/blueprint/%s".format(name), contents)
this
}
/** /**
* Add a Blueprint XML file to our test bundle * Add a Blueprint XML file to our test bundle
*/ */
def withBlueprintFile(contents: URL): BundleDescriptorBuilder = withBlueprintFile(filename(contents), contents) def withBlueprintFile(contents: URL): BundleDescriptorBuilder = {
val filename = contents.getFile.split("/").last
withBlueprintFile(filename, contents)
}
/** /**
* Add a Bundle activator to our test bundle * Add a Bundle activator to our test bundle
*/ */
def withActivator(activator: Class[_ <: BundleActivator]): BundleDescriptorBuilder = def withActivator(activator: Class[_ <: BundleActivator]): BundleDescriptorBuilder = {
returnBuilder(tinybundle.set(Constants.BUNDLE_ACTIVATOR, activator.getName)) tinybundle.set(Constants.BUNDLE_ACTIVATOR, activator.getName)
private def returnBuilder(block: Unit) = {
block
this this
} }
@ -122,11 +115,7 @@ class BundleDescriptorBuilder(name: String) {
*/ */
def build: BundleDescriptor = { def build: BundleDescriptor = {
val file: File = tinybundleToJarFile(name) val file: File = tinybundleToJarFile(name)
new BundleDescriptor(getClass().getClassLoader(), new URL("jar:" + file.toURI().toString() + "!/"), extractHeaders(file))
new BundleDescriptor(
getClass().getClassLoader(),
new URL("jar:" + file.toURI().toString() + "!/"),
extractHeaders(file))
} }
def extractHeaders(file: File): HashMap[String, String] = { def extractHeaders(file: File): HashMap[String, String] = {
@ -134,12 +123,9 @@ class BundleDescriptorBuilder(name: String) {
val jis = new JarInputStream(new FileInputStream(file)) val jis = new JarInputStream(new FileInputStream(file))
try { try {
for (entry jis.getManifest().getMainAttributes().entrySet()) { for (entry jis.getManifest().getMainAttributes().entrySet())
headers.put(entry.getKey().toString(), entry.getValue().toString()) headers.put(entry.getKey().toString(), entry.getValue().toString())
} } finally jis.close()
} finally {
jis.close()
}
headers headers
} }
@ -147,14 +133,9 @@ class BundleDescriptorBuilder(name: String) {
def tinybundleToJarFile(name: String): File = { def tinybundleToJarFile(name: String): File = {
val file = new File("target/%s-%tQ.jar".format(name, new Date())) val file = new File("target/%s-%tQ.jar".format(name, new Date()))
val fos = new FileOutputStream(file) val fos = new FileOutputStream(file)
try { try copy(tinybundle.build(), fos) finally fos.close()
copy(tinybundle.build(), fos)
} finally {
fos.close()
}
file file
} }
private[this] def filename(url: URL) = url.getFile.split("/").last
} }

View file

@ -43,7 +43,7 @@ object AkkaBuild extends Build {
sphinxLatex <<= sphinxLatex in LocalProject(docs.id), sphinxLatex <<= sphinxLatex in LocalProject(docs.id),
sphinxPdf <<= sphinxPdf in LocalProject(docs.id) sphinxPdf <<= sphinxPdf in LocalProject(docs.id)
), ),
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, samples, tutorials, docs) aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, samples, tutorials, osgi, osgiAries, docs)
) )
lazy val actor = Project( lazy val actor = Project(
@ -299,7 +299,7 @@ object AkkaBuild extends Build {
id = "akka-docs", id = "akka-docs",
base = file("akka-docs"), base = file("akka-docs"),
dependencies = Seq(actor, testkit % "test->test", mailboxesCommon % "compile;test->test", dependencies = Seq(actor, testkit % "test->test", mailboxesCommon % "compile;test->test",
remote, cluster, slf4j, agent, transactor, fileMailbox, zeroMQ, camel), remote, cluster, slf4j, agent, transactor, fileMailbox, zeroMQ, camel, osgi, osgiAries),
settings = defaultSettings ++ Sphinx.settings ++ Seq( settings = defaultSettings ++ Sphinx.settings ++ Seq(
unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get },
libraryDependencies ++= Dependencies.docs, libraryDependencies ++= Dependencies.docs,
@ -560,13 +560,9 @@ object OSGi {
val mailboxesCommon = exports(Seq("akka.actor.mailbox.*")) val mailboxesCommon = exports(Seq("akka.actor.mailbox.*"))
val osgi = exports(Seq("akka.osgi")) ++ Seq( val osgi = exports(Seq("akka.osgi")) ++ Seq(OsgiKeys.privatePackage := Seq("akka.osgi.impl"))
OsgiKeys.privatePackage := Seq("akka.osgi.impl")
)
val osgiAries = exports() ++ Seq( val osgiAries = exports() ++ Seq(OsgiKeys.privatePackage := Seq("akka.osgi.aries.*"))
OsgiKeys.privatePackage := Seq("akka.osgi.aries.*")
)
val remote = exports(Seq("akka.remote.*", "akka.routing.*", "akka.serialization.*")) val remote = exports(Seq("akka.remote.*", "akka.routing.*", "akka.serialization.*"))