diff --git a/akka-actors/pom.xml b/akka-actors/pom.xml index 91ab205c95..cb76e1fbb5 100644 --- a/akka-actors/pom.xml +++ b/akka-actors/pom.xml @@ -22,6 +22,11 @@ se.scalablesolutions.akka 0.6 + + akka-util + se.scalablesolutions.akka + 0.6 + org.scala-lang scala-library diff --git a/akka-actors/src/main/scala/actor/ActiveObject.scala b/akka-actors/src/main/scala/actor/ActiveObject.scala index cb22fecbc1..7f2d2f8fa3 100644 --- a/akka-actors/src/main/scala/actor/ActiveObject.scala +++ b/akka-actors/src/main/scala/actor/ActiveObject.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor import java.lang.reflect.{InvocationTargetException, Method} import java.net.InetSocketAddress -import kernel.reactor.{MessageDispatcher, FutureResult} -import kernel.nio.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply} -import kernel.nio.{RemoteProtocolBuilder, RemoteClient, RemoteServer, RemoteRequestIdFactory} -import kernel.config.ScalaConfig._ -import kernel.util._ +import reactor.{MessageDispatcher, FutureResult} +import nio.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply} +import nio.{RemoteProtocolBuilder, RemoteClient, RemoteServer, RemoteRequestIdFactory} +import config.ScalaConfig._ +import util._ import serialization.Serializer import org.codehaus.aspectwerkz.intercept.{Advisable, AroundAdvice, Advice} @@ -113,15 +113,15 @@ class ActiveObjectFactory { ActiveObject.newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout) } - private[kernel] def newInstance[T](target: Class[T], actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { + private[akka] def newInstance[T](target: Class[T], actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { ActiveObject.newInstance(target, actor, remoteAddress, timeout) } - private[kernel] def newInstance[T](intf: Class[T], target: AnyRef, actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { + private[akka] def newInstance[T](intf: Class[T], target: AnyRef, actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { ActiveObject.newInstance(intf, target, actor, remoteAddress, timeout) } - private[kernel] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor = + private[akka] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor = ActiveObject.supervise(restartStrategy, components) /* @@ -219,7 +219,7 @@ object ActiveObject { newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout) } - private[kernel] def newInstance[T](target: Class[T], actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { + private[akka] def newInstance[T](target: Class[T], actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { //if (getClass.getClassLoader.getResourceAsStream("META-INF/aop.xml") != null) println("000000000000000000000 FOUND AOP") if (remoteAddress.isDefined) actor.makeRemote(remoteAddress.get) val proxy = Proxy.newInstance(target, false, true) @@ -230,7 +230,7 @@ object ActiveObject { proxy.asInstanceOf[T] } - private[kernel] def newInstance[T](intf: Class[T], target: AnyRef, actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { + private[akka] def newInstance[T](intf: Class[T], target: AnyRef, actor: Dispatcher, remoteAddress: Option[InetSocketAddress], timeout: Long): T = { //if (getClass.getClassLoader.getResourceAsStream("META-INF/aop.xml") != null) println("000000000000000000000 FOUND AOP") if (remoteAddress.isDefined) actor.makeRemote(remoteAddress.get) val proxy = Proxy.newInstance(Array(intf), Array(target), false, true) @@ -242,7 +242,7 @@ object ActiveObject { } - private[kernel] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor = { + private[akka] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor = { object factory extends SupervisorFactory { override def getSupervisorConfig = SupervisorConfig(restartStrategy, components) } @@ -364,7 +364,7 @@ sealed class ActiveObjectAspect { * * @author Jonas Bonér */ -@serializable private[kernel] case class Invocation(val joinpoint: JoinPoint, val isOneWay: Boolean) { +@serializable private[akka] case class Invocation(val joinpoint: JoinPoint, val isOneWay: Boolean) { override def toString: String = synchronized { "Invocation [joinpoint: " + joinpoint.toString + ", isOneWay: " + isOneWay + "]" @@ -390,7 +390,7 @@ sealed class ActiveObjectAspect { * * @author Jonas Bonér */ -private[kernel] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends Actor { +private[akka] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends Actor { private val ZERO_ITEM_CLASS_ARRAY = Array[Class[_]]() private val ZERO_ITEM_OBJECT_ARRAY = Array[Object[_]]() diff --git a/akka-actors/src/main/scala/actor/Actor.scala b/akka-actors/src/main/scala/actor/Actor.scala index cf4dc132ca..5e0191483a 100644 --- a/akka-actors/src/main/scala/actor/Actor.scala +++ b/akka-actors/src/main/scala/actor/Actor.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor import com.google.protobuf.ByteString import java.net.InetSocketAddress @@ -42,8 +42,9 @@ class ActorMessageInvoker(val actor: Actor) extends MessageInvoker { * @author Jonas Bonér */ object Actor { - val TIMEOUT = Kernel.config.getInt("akka.actor.timeout", 5000) - val SERIALIZE_MESSAGES = Kernel.config.getBool("akka.actor.serialize-messages", false) + import Config._ + val TIMEOUT = config.getInt("akka.actor.timeout", 5000) + val SERIALIZE_MESSAGES = config.getBool("akka.actor.serialize-messages", false) } /** @@ -60,7 +61,7 @@ trait Actor extends Logging with TransactionManagement { private var config: Option[AnyRef] = None @volatile protected[this] var isTransactional = false @volatile protected[this] var remoteAddress: Option[InetSocketAddress] = None - @volatile protected[kernel] var supervisor: Option[Actor] = None + @volatile protected[akka] var supervisor: Option[Actor] = None protected[Actor] var mailbox: MessageQueue = _ protected[this] var senderFuture: Option[CompletableFutureResult] = None protected[this] val linkedActors = new CopyOnWriteArraySet[Actor] @@ -99,7 +100,7 @@ trait Actor extends Logging with TransactionManagement { * .buildThreadPool * */ - protected[kernel] var dispatcher: MessageDispatcher = { + protected[akka] var dispatcher: MessageDispatcher = { val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher(getClass.getName) mailbox = dispatcher.messageQueue dispatcher.registerHandler(this, new ActorMessageInvoker(this)) @@ -454,7 +455,7 @@ trait Actor extends Logging with TransactionManagement { /** * Callback for the dispatcher. E.g. single entry point to the user code and all protected[this] methods */ - private[kernel] def invoke(messageHandle: MessageInvocation) = synchronized { + private[akka] def invoke(messageHandle: MessageInvocation) = synchronized { if (TransactionManagement.isTransactionalityEnabled) transactionalDispatch(messageHandle) else dispatch(messageHandle) } @@ -577,7 +578,7 @@ trait Actor extends Logging with TransactionManagement { } } - private[kernel] def registerSupervisorAsRemoteActor: Option[String] = synchronized { + private[akka] def registerSupervisorAsRemoteActor: Option[String] = synchronized { if (supervisor.isDefined) { RemoteClient.clientFor(remoteAddress.get).registerSupervisorForActor(this) Some(supervisor.get.uuid) @@ -585,7 +586,7 @@ trait Actor extends Logging with TransactionManagement { } - private[kernel] def swapDispatcher(disp: MessageDispatcher) = synchronized { + private[akka] def swapDispatcher(disp: MessageDispatcher) = synchronized { dispatcher = disp mailbox = dispatcher.messageQueue dispatcher.registerHandler(this, new ActorMessageInvoker(this)) diff --git a/akka-actors/src/main/scala/actor/ActorRegistry.scala b/akka-actors/src/main/scala/actor/ActorRegistry.scala index 3f4275cf9e..2a534918d1 100755 --- a/akka-actors/src/main/scala/actor/ActorRegistry.scala +++ b/akka-actors/src/main/scala/actor/ActorRegistry.scala @@ -1,33 +1,33 @@ -/** - * Copyright (C) 2009 Scalable Solutions. - */ - -package se.scalablesolutions.akka.kernel.actor - -import kernel.util.Logging - -import scala.collection.jcl.HashMap - -/** - * Registry holding all actor instances, mapped by class.. - * - * @author Jonas Bonér - */ -object ActorRegistry extends Logging { - private val actors = new HashMap[String, List[Actor]] - - def actorsFor(clazz: Class[_]): List[Actor] = synchronized { - actors.get(clazz.getName) match { - case None => Nil - case Some(instances) => instances - } - } - - def register(actor: Actor) = synchronized { - val name = actor.getClass.getName - actors.get(name) match { - case Some(instances) => actors + (name -> (actor :: instances)) - case None => actors + (name -> (actor :: Nil)) - } - } -} +/** + * Copyright (C) 2009 Scalable Solutions. + */ + +package se.scalablesolutions.akka.actor + +import util.Logging + +import scala.collection.jcl.HashMap + +/** + * Registry holding all actor instances, mapped by class.. + * + * @author Jonas Bonér + */ +object ActorRegistry extends Logging { + private val actors = new HashMap[String, List[Actor]] + + def actorsFor(clazz: Class[_]): List[Actor] = synchronized { + actors.get(clazz.getName) match { + case None => Nil + case Some(instances) => instances + } + } + + def register(actor: Actor) = synchronized { + val name = actor.getClass.getName + actors.get(name) match { + case Some(instances) => actors + (name -> (actor :: instances)) + case None => actors + (name -> (actor :: Nil)) + } + } +} diff --git a/akka-actors/src/main/scala/util/Scheduler.scala b/akka-actors/src/main/scala/actor/Scheduler.scala similarity index 96% rename from akka-actors/src/main/scala/util/Scheduler.scala rename to akka-actors/src/main/scala/actor/Scheduler.scala index 2d7bfed445..d9f4ae0c48 100644 --- a/akka-actors/src/main/scala/util/Scheduler.scala +++ b/akka-actors/src/main/scala/actor/Scheduler.scala @@ -11,10 +11,10 @@ * limitations under the License. */ -package se.scalablesolutions.akka.kernel.util +package se.scalablesolutions.akka.util import java.util.concurrent._ -import kernel.actor.{OneForOneStrategy, Actor} +import actor.{OneForOneStrategy, Actor} import org.scala_tools.javautils.Imports._ diff --git a/akka-actors/src/main/scala/actor/Supervisor.scala b/akka-actors/src/main/scala/actor/Supervisor.scala index 0b9f1dfc4c..e183cfcc96 100644 --- a/akka-actors/src/main/scala/actor/Supervisor.scala +++ b/akka-actors/src/main/scala/actor/Supervisor.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor -import kernel.util.Logging -import kernel.config.ScalaConfig._ -import kernel.config.{ConfiguratorRepository, Configurator} -import kernel.util.Helpers._ +import util.Logging +import config.ScalaConfig._ +import config.{ConfiguratorRepository, Configurator} +import util.Helpers._ import java.util.concurrent.ConcurrentHashMap diff --git a/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala b/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala index d1b17782a8..9c8e57578d 100644 --- a/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala +++ b/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config +package se.scalablesolutions.akka.config import com.google.inject._ import ScalaConfig._ -import kernel.actor.{Supervisor, ActiveObjectFactory, Dispatcher} -import kernel.util.Logging +import akka.actor.{Supervisor, ActiveObjectFactory, Dispatcher} +import akka.util.Logging -import org.apache.camel.impl.{DefaultCamelContext} -import org.apache.camel.{CamelContext, Endpoint, Routes} +//import org.apache.camel.impl.{DefaultCamelContext} +//import org.apache.camel.{CamelContext, Endpoint, Routes} import scala.collection.mutable.HashMap @@ -21,8 +21,8 @@ import java.lang.reflect.Method /** * @author Jonas Bonér */ -class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with CamelConfigurator with Logging { - val AKKA_CAMEL_ROUTING_SCHEME = "akka" +class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with Logging { // with CamelConfigurator { + //val AKKA_CAMEL_ROUTING_SCHEME = "akka" private var injector: Injector = _ private var supervisor: Option[Supervisor] = None @@ -33,7 +33,7 @@ class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with CamelC private var configRegistry = new HashMap[Class[_], Component] // TODO is configRegistry needed? private var activeObjectRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]] private var activeObjectFactory = new ActiveObjectFactory - private var camelContext = new DefaultCamelContext + //private var camelContext = new DefaultCamelContext private var modules = new java.util.ArrayList[Module] private var methodToUriRegistry = new HashMap[Method, String] @@ -65,7 +65,7 @@ class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with CamelC if (c.intf.isDefined) c.intf.get else c.target } - + /* override def getRoutingEndpoint(uri: String): Endpoint = synchronized { camelContext.getEndpoint(uri) } @@ -77,7 +77,7 @@ class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with CamelC override def getRoutingEndpoints(uri: String): java.util.Collection[Endpoint] = synchronized { camelContext.getEndpoints(uri) } - + */ override def configure(restartStrategy: RestartStrategy, components: List[Component]): ActiveObjectConfigurator = synchronized { this.restartStrategy = restartStrategy @@ -155,14 +155,14 @@ class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with CamelC modules.add(module) this } - + /* override def addRoutes(routes: Routes): ActiveObjectConfigurator = synchronized { camelContext.addRoutes(routes) this } override def getCamelContext: CamelContext = camelContext - + */ def getGuiceModules: java.util.List[Module] = modules def reset = synchronized { @@ -172,21 +172,21 @@ class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurator with CamelC methodToUriRegistry = new HashMap[Method, String] injector = null restartStrategy = null - camelContext = new DefaultCamelContext + //camelContext = new DefaultCamelContext } def stop = synchronized { - camelContext.stop + //camelContext.stop if (supervisor.isDefined) supervisor.get.stop } - def registerMethodForUri(method: Method, componentName: String) = - methodToUriRegistry += method -> buildUri(method, componentName) +// def registerMethodForUri(method: Method, componentName: String) = +// methodToUriRegistry += method -> buildUri(method, componentName) - def lookupUriFor(method: Method): String = - methodToUriRegistry.getOrElse(method, throw new IllegalStateException("Could not find URI for method [" + method.getName + "]")) +// def lookupUriFor(method: Method): String = +// methodToUriRegistry.getOrElse(method, throw new IllegalStateException("Could not find URI for method [" + method.getName + "]")) - def buildUri(method: Method, componentName: String): String = - AKKA_CAMEL_ROUTING_SCHEME + ":" + componentName + "." + method.getName +// def buildUri(method: Method, componentName: String): String = +// AKKA_CAMEL_ROUTING_SCHEME + ":" + componentName + "." + method.getName } \ No newline at end of file diff --git a/akka-actors/src/main/scala/config/ActiveObjectManager.scala b/akka-actors/src/main/scala/config/ActiveObjectManager.scala index 2545d44b29..f5d49d426a 100644 --- a/akka-actors/src/main/scala/config/ActiveObjectManager.scala +++ b/akka-actors/src/main/scala/config/ActiveObjectManager.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config +package se.scalablesolutions.akka.config -import akka.kernel.config.JavaConfig._ +import JavaConfig._ import com.google.inject._ import java.util._ -import org.apache.camel.impl.{JndiRegistry, DefaultCamelContext} -import org.apache.camel.{Endpoint, Routes} +//import org.apache.camel.impl.{JndiRegistry, DefaultCamelContext} +//import org.apache.camel.{Endpoint, Routes} /** * @@ -49,10 +49,10 @@ class ActiveObjectManager { this } - def addRoutes(routes: Routes): ActiveObjectManager = { - INSTANCE.addRoutes(routes) - this - } + //def addRoutes(routes: Routes): ActiveObjectManager = { + // INSTANCE.addRoutes(routes) + // this + // } def getComponentInterfaces: List[Class[_]] = { @@ -63,11 +63,11 @@ class ActiveObjectManager { def getExternalDependency[T](clazz: Class[T]): T = INSTANCE.getExternalDependency(clazz) - def getRoutingEndpoint(uri: String): Endpoint = INSTANCE.getRoutingEndpoint(uri) + //def getRoutingEndpoint(uri: String): Endpoint = INSTANCE.getRoutingEndpoint(uri) - def getRoutingEndpoints: java.util.Collection[Endpoint] = INSTANCE.getRoutingEndpoints + //def getRoutingEndpoints: java.util.Collection[Endpoint] = INSTANCE.getRoutingEndpoints - def getRoutingEndpoints(uri: String): java.util.Collection[Endpoint] = INSTANCE.getRoutingEndpoints(uri) + //def getRoutingEndpoints(uri: String): java.util.Collection[Endpoint] = INSTANCE.getRoutingEndpoints(uri) def getGuiceModules: List[Module] = INSTANCE.getGuiceModules diff --git a/akka-actors/src/main/scala/config/Config.scala b/akka-actors/src/main/scala/config/Config.scala index 0df243a338..f9a6f25d1f 100644 --- a/akka-actors/src/main/scala/config/Config.scala +++ b/akka-actors/src/main/scala/config/Config.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config +package se.scalablesolutions.akka.config import reflect.BeanProperty -import kernel.actor.Actor -import kernel.reactor.MessageDispatcher +import actor.Actor +import reactor.MessageDispatcher /** * Configuration classes - not to be used as messages. @@ -95,7 +95,7 @@ object JavaConfig { @BeanProperty val scheme: FailOverScheme, @BeanProperty val maxNrOfRetries: Int, @BeanProperty val withinTimeRange: Int) extends ConfigElement { - def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.RestartStrategy( + def transform = se.scalablesolutions.akka.config.ScalaConfig.RestartStrategy( scheme.transform, maxNrOfRetries, withinTimeRange) } @@ -103,35 +103,35 @@ object JavaConfig { def this(scope: Scope, shutdownTime: Int) = this(scope, shutdownTime, null) def transform = { val callbackOption = if (callbacks == null) None else Some(callbacks.transform) - se.scalablesolutions.akka.kernel.config.ScalaConfig.LifeCycle(scope.transform, shutdownTime, callbackOption) + se.scalablesolutions.akka.config.ScalaConfig.LifeCycle(scope.transform, shutdownTime, callbackOption) } } class RestartCallbacks(@BeanProperty val preRestart: String, @BeanProperty val postRestart: String) { - def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.RestartCallbacks(preRestart, postRestart) + def transform = se.scalablesolutions.akka.config.ScalaConfig.RestartCallbacks(preRestart, postRestart) } abstract class Scope extends ConfigElement { - def transform: se.scalablesolutions.akka.kernel.config.ScalaConfig.Scope + def transform: se.scalablesolutions.akka.config.ScalaConfig.Scope } class Permanent extends Scope { - override def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.Permanent + override def transform = se.scalablesolutions.akka.config.ScalaConfig.Permanent } class Transient extends Scope { - override def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.Transient + override def transform = se.scalablesolutions.akka.config.ScalaConfig.Transient } class Temporary extends Scope { - override def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.Temporary + override def transform = se.scalablesolutions.akka.config.ScalaConfig.Temporary } abstract class FailOverScheme extends ConfigElement { - def transform: se.scalablesolutions.akka.kernel.config.ScalaConfig.FailOverScheme + def transform: se.scalablesolutions.akka.config.ScalaConfig.FailOverScheme } class AllForOne extends FailOverScheme { - override def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.AllForOne + override def transform = se.scalablesolutions.akka.config.ScalaConfig.AllForOne } class OneForOne extends FailOverScheme { - override def transform = se.scalablesolutions.akka.kernel.config.ScalaConfig.OneForOne + override def transform = se.scalablesolutions.akka.config.ScalaConfig.OneForOne } class RemoteAddress(@BeanProperty val hostname: String, @BeanProperty val port: Int) @@ -167,11 +167,11 @@ object JavaConfig { this(null, target, lifeCycle, timeout, dispatcher, remoteAddress) def transform = - se.scalablesolutions.akka.kernel.config.ScalaConfig.Component(intf, target, lifeCycle.transform, timeout, dispatcher, - if (remoteAddress != null) se.scalablesolutions.akka.kernel.config.ScalaConfig.RemoteAddress(remoteAddress.hostname, remoteAddress.port) else null) + se.scalablesolutions.akka.config.ScalaConfig.Component(intf, target, lifeCycle.transform, timeout, dispatcher, + if (remoteAddress != null) se.scalablesolutions.akka.config.ScalaConfig.RemoteAddress(remoteAddress.hostname, remoteAddress.port) else null) def newSupervised(actor: Actor) = - se.scalablesolutions.akka.kernel.config.ScalaConfig.Supervise(actor, lifeCycle.transform) + se.scalablesolutions.akka.config.ScalaConfig.Supervise(actor, lifeCycle.transform) } } \ No newline at end of file diff --git a/akka-actors/src/main/scala/config/Configuration.scala b/akka-actors/src/main/scala/config/Configuration.scala index d4e63123dc..5813cd37bf 100644 --- a/akka-actors/src/main/scala/config/Configuration.scala +++ b/akka-actors/src/main/scala/config/Configuration.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config +package se.scalablesolutions.akka.config /* import se.scalablesolutions.akka.kernel.{ActiveObject, ActiveObjectProxy} diff --git a/akka-actors/src/main/scala/config/Configurator.scala b/akka-actors/src/main/scala/config/Configurator.scala index bae832f459..c4a57e91a4 100644 --- a/akka-actors/src/main/scala/config/Configurator.scala +++ b/akka-actors/src/main/scala/config/Configurator.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config +package se.scalablesolutions.akka.config import ScalaConfig.{RestartStrategy, Component} diff --git a/akka-actors/src/main/scala/config/ConfiguratorRepository.scala b/akka-actors/src/main/scala/config/ConfiguratorRepository.scala index 249746914e..049bf0ed4f 100644 --- a/akka-actors/src/main/scala/config/ConfiguratorRepository.scala +++ b/akka-actors/src/main/scala/config/ConfiguratorRepository.scala @@ -2,36 +2,28 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config - -import javax.servlet.ServletContext +package se.scalablesolutions.akka.config import scala.collection.mutable.HashSet -import kernel.util.Logging +import util.Logging object ConfiguratorRepository extends Logging { private val configuration = new HashSet[Configurator] - // FIXME: cheating with only having one single, scope per ServletContext def registerConfigurator(conf: Configurator) = synchronized { configuration + conf } - def getConfiguratorsFor(ctx: ServletContext): List[Configurator] = synchronized { + def getConfigurators: List[Configurator] = synchronized { configuration.toList //configurations.getOrElse(ctx, throw new IllegalArgumentException("No configuration for servlet context [" + ctx + "]")) } } class ConfiguratorRepository extends Logging { - def registerConfigurator(conf: Configurator) = { - ConfiguratorRepository.registerConfigurator(conf) - } - - def getConfiguratorsFor(ctx: ServletContext): List[Configurator] = { - ConfiguratorRepository.getConfiguratorsFor(ctx) - } + def registerConfigurator(conf: Configurator) = ConfiguratorRepository.registerConfigurator(conf) + def getConfigurators: List[Configurator] = ConfiguratorRepository.getConfigurators } diff --git a/akka-actors/src/main/scala/nio/RemoteClient.scala b/akka-actors/src/main/scala/nio/RemoteClient.scala index 0ee67950e2..74e99b9fe8 100644 --- a/akka-actors/src/main/scala/nio/RemoteClient.scala +++ b/akka-actors/src/main/scala/nio/RemoteClient.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.nio +package se.scalablesolutions.akka.nio import java.net.InetSocketAddress import java.util.concurrent.{Executors, ConcurrentMap, ConcurrentHashMap} import protobuf.RemoteProtocol.{RemoteRequest, RemoteReply} -import kernel.actor.{Exit, Actor} -import kernel.reactor.{DefaultCompletableFutureResult, CompletableFutureResult} +import actor.{Exit, Actor} +import reactor.{DefaultCompletableFutureResult, CompletableFutureResult} import serialization.{Serializer, Serializable, SerializationProtocol} -import kernel.util.Logging +import util.Logging import org.jboss.netty.bootstrap.ClientBootstrap import org.jboss.netty.channel._ diff --git a/akka-actors/src/main/scala/nio/RemoteProtocolBuilder.scala b/akka-actors/src/main/scala/nio/RemoteProtocolBuilder.scala index b492a2fe0c..1c846a42cd 100644 --- a/akka-actors/src/main/scala/nio/RemoteProtocolBuilder.scala +++ b/akka-actors/src/main/scala/nio/RemoteProtocolBuilder.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.nio +package se.scalablesolutions.akka.nio import akka.serialization.Serializable.SBinary import com.google.protobuf.{Message, ByteString} diff --git a/akka-actors/src/main/scala/nio/RemoteServer.scala b/akka-actors/src/main/scala/nio/RemoteServer.scala index 76a7be6859..ccb7661674 100644 --- a/akka-actors/src/main/scala/nio/RemoteServer.scala +++ b/akka-actors/src/main/scala/nio/RemoteServer.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.nio +package se.scalablesolutions.akka.nio import java.lang.reflect.InvocationTargetException import java.net.InetSocketAddress import java.util.concurrent.{ConcurrentHashMap, Executors} -import kernel.actor._ -import kernel.util._ +import actor._ +import util._ import protobuf.RemoteProtocol import protobuf.RemoteProtocol.{RemoteReply, RemoteRequest} import serialization.{Serializer, Serializable, SerializationProtocol} @@ -31,7 +31,7 @@ class RemoteServer extends Logging { * @author Jonas Bonér */ object RemoteServer extends Logging { - import kernel.Kernel.config + import Config.config val HOSTNAME = config.getString("akka.remote.hostname", "localhost") val PORT = config.getInt("akka.remote.port", 9999) val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.connection-timeout", 1000) diff --git a/akka-actors/src/main/scala/nio/RequestReply.scala b/akka-actors/src/main/scala/nio/RequestReply.scala index b48732cf94..ce79653f06 100644 --- a/akka-actors/src/main/scala/nio/RequestReply.scala +++ b/akka-actors/src/main/scala/nio/RequestReply.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.nio +package se.scalablesolutions.akka.nio import java.util.concurrent.atomic.AtomicLong -import kernel.stm.Transaction -import kernel.util.HashCode +import stm.Transaction +import util.HashCode // FIXME: will not work - can clash with other host's requests - need te prepend with hostname object RemoteRequestIdFactory { diff --git a/akka-actors/src/main/scala/reactor/Dispatchers.scala b/akka-actors/src/main/scala/reactor/Dispatchers.scala index 5c4935bbd5..30846752b6 100644 --- a/akka-actors/src/main/scala/reactor/Dispatchers.scala +++ b/akka-actors/src/main/scala/reactor/Dispatchers.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor -import kernel.actor.Actor +import actor.Actor /** * Scala API. Dispatcher factory. diff --git a/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala b/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala index 65c71da8f8..90518dbd73 100644 --- a/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala +++ b/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala @@ -8,7 +8,7 @@ * * Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/]. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.{LinkedList, List} diff --git a/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala b/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala index 101f7cc0dd..c63237f12c 100644 --- a/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala +++ b/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.concurrent._ import locks.ReentrantLock diff --git a/akka-actors/src/main/scala/reactor/Future.scala b/akka-actors/src/main/scala/reactor/Future.scala index c81a88b264..c43b212932 100644 --- a/akka-actors/src/main/scala/reactor/Future.scala +++ b/akka-actors/src/main/scala/reactor/Future.scala @@ -5,7 +5,7 @@ /** * Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/]. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.concurrent.locks.{Lock, Condition, ReentrantLock} import java.util.concurrent.TimeUnit diff --git a/akka-actors/src/main/scala/reactor/ManagedActorScheduler.scala b/akka-actors/src/main/scala/reactor/ManagedActorScheduler.scala deleted file mode 100644 index a251557fc5..0000000000 --- a/akka-actors/src/main/scala/reactor/ManagedActorScheduler.scala +++ /dev/null @@ -1,68 +0,0 @@ -/** - * Copyright (C) 2009 Scalable Solutions. - */ - -package se.scalablesolutions.akka.kernel - -import kernel.util.Logging -import net.lag.logging.Logger - -import java.util.concurrent.Executors - -import scala.actors.{FJTaskScheduler2, Scheduler, IScheduler, Actor} - -// FIXME: add managing interface to this class using JMX -// FIXME: configure one instance per GenericServer - -class ManagedActorScheduler extends IScheduler with Logging { - protected var threadPoolSize = 10 - protected val threadPool = Executors.newFixedThreadPool(threadPoolSize) - - def execute(fun: => Unit): Unit = threadPool.execute(new Runnable { - def run = { - try { - fun - } catch { - case e => log.error("Actor scheduler", e) - } - } - }) - - def execute(task: Runnable) = threadPool.execute(new Runnable { - def run = { - try { - task.run - } catch { - case e => log.error("Actor scheduler", e) - } - } - }) - - def tick(a: Actor): Unit = {} - - def shutdown: Unit = threadPool.shutdown - - def onLockup(handler: () => Unit): Unit = {} - - def onLockup(millis: Int)(handler: () => Unit): Unit = {} - - def printActorDump: Unit = {} -} - -object GlobalManagedActorScheduler extends Logging { - @volatile private var isRegistered = false - private var threadPoolSize = 10 - - def register = synchronized { - if (!isRegistered) { - Scheduler.impl match { - case fj: FJTaskScheduler2 => - fj.snapshot - fj.shutdown - case _ => - } - Scheduler.impl = new ManagedActorScheduler - isRegistered = true - } - } -} \ No newline at end of file diff --git a/akka-actors/src/main/scala/reactor/MessageDispatcherBase.scala b/akka-actors/src/main/scala/reactor/MessageDispatcherBase.scala index 27ed887c7a..8a6670bdc4 100644 --- a/akka-actors/src/main/scala/reactor/MessageDispatcherBase.scala +++ b/akka-actors/src/main/scala/reactor/MessageDispatcherBase.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.{LinkedList, Queue, List} import java.util.concurrent.{TimeUnit, BlockingQueue} @@ -10,7 +10,7 @@ import java.util.HashMap abstract class MessageDispatcherBase(val name: String) extends MessageDispatcher { - //val CONCURRENT_MODE = kernel.Kernel.config.getBool("akka.actor.concurrent-mode", false) + //val CONCURRENT_MODE = Config.config.getBool("akka.actor.concurrent-mode", false) val MILLISECONDS = TimeUnit.MILLISECONDS val queue = new ReactiveMessageQueue(name) var blockingQueue: BlockingQueue[Runnable] = _ @@ -42,7 +42,7 @@ abstract class MessageDispatcherBase(val name: String) extends MessageDispatcher } class ReactiveMessageQueue(name: String) extends MessageQueue { - private[kernel] val queue: Queue[MessageInvocation] = new LinkedList[MessageInvocation] + private[akka] val queue: Queue[MessageInvocation] = new LinkedList[MessageInvocation] @volatile private var interrupted = false def append(handle: MessageInvocation) = queue.synchronized { diff --git a/akka-actors/src/main/scala/reactor/Reactor.scala b/akka-actors/src/main/scala/reactor/Reactor.scala index 9e4983a6f8..a4bb3e3784 100644 --- a/akka-actors/src/main/scala/reactor/Reactor.scala +++ b/akka-actors/src/main/scala/reactor/Reactor.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.List -import kernel.stm.Transaction -import kernel.util.HashCode +import stm.Transaction +import util.HashCode trait MessageQueue { def append(handle: MessageInvocation) diff --git a/akka-actors/src/main/scala/reactor/ThreadBasedDispatcher.scala b/akka-actors/src/main/scala/reactor/ThreadBasedDispatcher.scala index 28f9ca2761..aa04414169 100644 --- a/akka-actors/src/main/scala/reactor/ThreadBasedDispatcher.scala +++ b/akka-actors/src/main/scala/reactor/ThreadBasedDispatcher.scala @@ -2,18 +2,18 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.concurrent.LinkedBlockingQueue import java.util.Queue -import kernel.actor.{Actor, ActorMessageInvoker} +import actor.{Actor, ActorMessageInvoker} /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * @author Jonas Bonér */ -class ThreadBasedDispatcher private[kernel] (val name: String, val messageHandler: MessageInvoker) extends MessageDispatcher { +class ThreadBasedDispatcher private[akka] (val name: String, val messageHandler: MessageInvoker) extends MessageDispatcher { def this(actor: Actor) = this(actor.getClass.getName, new ActorMessageInvoker(actor)) private val queue = new BlockingMessageQueue(name) diff --git a/akka-stm/src/main/scala/ChangeSet.scala b/akka-actors/src/main/scala/stm/ChangeSet.scala similarity index 56% rename from akka-stm/src/main/scala/ChangeSet.scala rename to akka-actors/src/main/scala/stm/ChangeSet.scala index 5910714da8..440df8c18d 100644 --- a/akka-stm/src/main/scala/ChangeSet.scala +++ b/akka-actors/src/main/scala/stm/ChangeSet.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.stm +package se.scalablesolutions.akka.stm -import kernel.state.{Transactional, TransactionalMap} -import kernel.util.Helpers.ReadWriteLock +import state.{Transactional, TransactionalMap} +import util.Helpers.ReadWriteLock import scala.collection.immutable.HashSet @serializable @@ -13,14 +13,14 @@ class ChangeSet { private val lock = new ReadWriteLock private var transactionalItems: Set[Transactional] = new HashSet - private[kernel] def +(item: Transactional) = lock.withWriteLock { + private[akka] def +(item: Transactional) = lock.withWriteLock { transactionalItems += item } - private[kernel] def items: List[Transactional] = lock.withReadLock { + private[akka] def items: List[Transactional] = lock.withReadLock { transactionalItems.toList.asInstanceOf[List[Transactional]] } - private[kernel] def clear = lock.withWriteLock { + private[akka] def clear = lock.withWriteLock { transactionalItems = new HashSet } diff --git a/akka-stm/src/main/scala/HashTrie.scala b/akka-actors/src/main/scala/stm/HashTrie.scala similarity index 100% rename from akka-stm/src/main/scala/HashTrie.scala rename to akka-actors/src/main/scala/stm/HashTrie.scala diff --git a/akka-actors/src/main/scala/util/ResultOrFailure.scala b/akka-actors/src/main/scala/stm/ResultOrFailure.scala similarity index 91% rename from akka-actors/src/main/scala/util/ResultOrFailure.scala rename to akka-actors/src/main/scala/stm/ResultOrFailure.scala index 69d3f90b38..f768add9c0 100644 --- a/akka-actors/src/main/scala/util/ResultOrFailure.scala +++ b/akka-actors/src/main/scala/stm/ResultOrFailure.scala @@ -1,60 +1,60 @@ -/** - * Copyright (C) 2009 Scalable Solutions. - */ - -package se.scalablesolutions.akka.kernel.util - -import kernel.stm.Transaction - -/** - * Reference that can hold either a typed value or an exception. - * - * Usage: - *
- * scala> ResultOrFailure(1)
- * res0: ResultOrFailure[Int] = ResultOrFailure@a96606
- *  
- * scala> res0()
- * res1: Int = 1
- *
- * scala> res0() = 3
- *
- * scala> res0()
- * res3: Int = 3
- * 
- * scala> res0() = { println("Hello world"); 3}
- * Hello world
- *
- * scala> res0()
- * res5: Int = 3
- *  
- * scala> res0() = error("Lets see what happens here...")
- *
- * scala> res0()
- * java.lang.RuntimeException: Lets see what happens here...
- * 	at ResultOrFailure.apply(RefExcept.scala:11)
- * 	at .(:6)
- * 	at .()
- * 	at Re...
- * 
- * - * @author Jonas Bonér - */ -class ResultOrFailure[Payload](payload: Payload, val tx: Option[Transaction]) { - private[this] var contents: Either[Throwable, Payload] = Right(payload) - - def update(value: => Payload) = { - contents = try { Right(value) } catch { case (e : Throwable) => Left(e) } - } - - def apply() = contents match { - case Right(payload) => payload - case Left(e) => throw e - } - - override def toString(): String = "ResultOrFailure[" + contents + "]" -} -object ResultOrFailure { - def apply[Payload](payload: Payload, tx: Option[Transaction]) = new ResultOrFailure(payload, tx) - def apply[AnyRef](tx: Option[Transaction]) = new ResultOrFailure(new Object, tx) -} +/** + * Copyright (C) 2009 Scalable Solutions. + */ + +package se.scalablesolutions.akka.util + +import stm.Transaction + +/** + * Reference that can hold either a typed value or an exception. + * + * Usage: + *
+ * scala> ResultOrFailure(1)
+ * res0: ResultOrFailure[Int] = ResultOrFailure@a96606
+ *  
+ * scala> res0()
+ * res1: Int = 1
+ *
+ * scala> res0() = 3
+ *
+ * scala> res0()
+ * res3: Int = 3
+ * 
+ * scala> res0() = { println("Hello world"); 3}
+ * Hello world
+ *
+ * scala> res0()
+ * res5: Int = 3
+ *  
+ * scala> res0() = error("Lets see what happens here...")
+ *
+ * scala> res0()
+ * java.lang.RuntimeException: Lets see what happens here...
+ * 	at ResultOrFailure.apply(RefExcept.scala:11)
+ * 	at .(:6)
+ * 	at .()
+ * 	at Re...
+ * 
+ * + * @author Jonas Bonér + */ +class ResultOrFailure[Payload](payload: Payload, val tx: Option[Transaction]) { + private[this] var contents: Either[Throwable, Payload] = Right(payload) + + def update(value: => Payload) = { + contents = try { Right(value) } catch { case (e : Throwable) => Left(e) } + } + + def apply() = contents match { + case Right(payload) => payload + case Left(e) => throw e + } + + override def toString(): String = "ResultOrFailure[" + contents + "]" +} +object ResultOrFailure { + def apply[Payload](payload: Payload, tx: Option[Transaction]) = new ResultOrFailure(payload, tx) + def apply[AnyRef](tx: Option[Transaction]) = new ResultOrFailure(new Object, tx) +} diff --git a/akka-stm/src/main/scala/Transaction.scala b/akka-actors/src/main/scala/stm/Transaction.scala similarity index 97% rename from akka-stm/src/main/scala/Transaction.scala rename to akka-actors/src/main/scala/stm/Transaction.scala index b72eb7c2b2..bcd944de23 100644 --- a/akka-stm/src/main/scala/Transaction.scala +++ b/akka-actors/src/main/scala/stm/Transaction.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.stm +package se.scalablesolutions.akka.stm import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} -import kernel.state.Transactional -import kernel.util.Logging +import state.Transactional +import util.Logging @serializable sealed abstract class TransactionStatus object TransactionStatus { @@ -134,7 +134,7 @@ object TransactionIdFactory { throw new IllegalStateException("Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString) // For reinitialize transaction after sending it over the wire - private[kernel] def reinit = synchronized { + private[akka] def reinit = synchronized { import net.lag.logging.{Logger, Level} if (log == null) { log = Logger.get(this.getClass.getName) diff --git a/akka-stm/src/main/scala/TransactionManagement.scala b/akka-actors/src/main/scala/stm/TransactionManagement.scala similarity index 84% rename from akka-stm/src/main/scala/TransactionManagement.scala rename to akka-actors/src/main/scala/stm/TransactionManagement.scala index e67f4ce861..9782b352cf 100644 --- a/akka-stm/src/main/scala/TransactionManagement.scala +++ b/akka-actors/src/main/scala/stm/TransactionManagement.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.stm +package se.scalablesolutions.akka.stm import java.util.concurrent.atomic.AtomicBoolean -import kernel.reactor.MessageInvocation -import kernel.util.Logging +import reactor.MessageInvocation +import util.Logging import org.codehaus.aspectwerkz.proxy.Uuid // FIXME is java.util.UUID better? class TransactionRollbackException(msg: String) extends RuntimeException(msg) @@ -17,16 +17,17 @@ class TransactionAwareWrapperException(val cause: Throwable, val tx: Option[Tran } object TransactionManagement { - val TIME_WAITING_FOR_COMPLETION = kernel.Kernel.config.getInt("akka.stm.wait-for-completion", 100) - val NR_OF_TIMES_WAITING_FOR_COMPLETION = kernel.Kernel.config.getInt("akka.stm.wait-nr-of-times", 3) - val TRANSACTION_ENABLED = new AtomicBoolean(kernel.Kernel.config.getBool("akka.stm.service", true)) + import Config._ + val TIME_WAITING_FOR_COMPLETION = config.getInt("akka.stm.wait-for-completion", 100) + val NR_OF_TIMES_WAITING_FOR_COMPLETION = config.getInt("akka.stm.wait-nr-of-times", 3) + val TRANSACTION_ENABLED = new AtomicBoolean(config.getBool("akka.stm.service", true)) // FIXME reenable 'akka.stm.restart-on-collision' when new STM is in place - val RESTART_TRANSACTION_ON_COLLISION = false //kernel.Kernel.config.getBool("akka.stm.restart-on-collision", true) + val RESTART_TRANSACTION_ON_COLLISION = false //akka.Kernel.config.getBool("akka.stm.restart-on-collision", true) def isTransactionalityEnabled = TRANSACTION_ENABLED.get def disableTransactions = TRANSACTION_ENABLED.set(false) - private[kernel] val threadBoundTx: ThreadLocal[Option[Transaction]] = new ThreadLocal[Option[Transaction]]() { + private[akka] val threadBoundTx: ThreadLocal[Option[Transaction]] = new ThreadLocal[Option[Transaction]]() { override protected def initialValue: Option[Transaction] = None } } @@ -38,7 +39,7 @@ trait TransactionManagement extends Logging { protected[this] var messageToReschedule: Option[MessageInvocation] = None import TransactionManagement.threadBoundTx - private[kernel] var activeTx: Option[Transaction] = None + private[akka] var activeTx: Option[Transaction] = None protected def startNewTransaction: Option[Transaction] = { val newTx = new Transaction diff --git a/akka-stm/src/main/scala/TransactionWatcher.scala b/akka-actors/src/main/scala/stm/TransactionWatcher.scala similarity index 98% rename from akka-stm/src/main/scala/TransactionWatcher.scala rename to akka-actors/src/main/scala/stm/TransactionWatcher.scala index 755a75f54d..2dd966c7ba 100644 --- a/akka-stm/src/main/scala/TransactionWatcher.scala +++ b/akka-actors/src/main/scala/stm/TransactionWatcher.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.stm +package se.scalablesolutions.akka.stm /* import kernel.util.Logging diff --git a/akka-stm/src/main/scala/TransactionalState.scala b/akka-actors/src/main/scala/stm/TransactionalState.scala similarity index 92% rename from akka-stm/src/main/scala/TransactionalState.scala rename to akka-actors/src/main/scala/stm/TransactionalState.scala index 266070f15b..c671a72ea5 100644 --- a/akka-stm/src/main/scala/TransactionalState.scala +++ b/akka-actors/src/main/scala/stm/TransactionalState.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.state +package se.scalablesolutions.akka.state -import kernel.stm.TransactionManagement +import stm.TransactionManagement import akka.collection._ import org.codehaus.aspectwerkz.proxy.Uuid @@ -44,9 +44,9 @@ trait Transactional { // FIXME: won't work across the cluster val uuid = Uuid.newUuid.toString - private[kernel] def begin - private[kernel] def commit - private[kernel] def rollback + private[akka] def begin + private[akka] def commit + private[akka] def rollback protected def verifyTransaction = { val cflowTx = TransactionManagement.threadBoundTx.get @@ -77,8 +77,8 @@ trait TransactionalMap[K, V] extends Transactional with scala.collection.mutable * @author Jonas Bonér */ class InMemoryTransactionalMap[K, V] extends TransactionalMap[K, V] { - protected[kernel] var state = new HashTrie[K, V] - protected[kernel] var snapshot = state + protected[akka] var state = new HashTrie[K, V] + protected[akka] var snapshot = state // ---- For Transactional ---- override def begin = snapshot = state @@ -159,8 +159,8 @@ abstract class TransactionalVector[T] extends Transactional with RandomAccessSeq * @author Jonas Bonér */ class InMemoryTransactionalVector[T] extends TransactionalVector[T] { - private[kernel] var state: Vector[T] = EmptyVector - private[kernel] var snapshot = state + private[akka] var state: Vector[T] = EmptyVector + private[akka] var snapshot = state def add(elem: T) = { verifyTransaction @@ -214,8 +214,8 @@ class InMemoryTransactionalVector[T] extends TransactionalVector[T] { * @author Jonas Bonér */ class TransactionalRef[T] extends Transactional { - private[kernel] var ref: Option[T] = None - private[kernel] var snapshot: Option[T] = None + private[akka] var ref: Option[T] = None + private[akka] var snapshot: Option[T] = None override def begin = if (ref.isDefined) snapshot = Some(ref.get) diff --git a/akka-stm/src/main/scala/Vector.scala b/akka-actors/src/main/scala/stm/Vector.scala similarity index 100% rename from akka-stm/src/main/scala/Vector.scala rename to akka-actors/src/main/scala/stm/Vector.scala diff --git a/akka-actors/src/test/scala/ActorSpec.scala b/akka-actors/src/test/scala/ActorSpec.scala index 5977301e7a..74ebd13f25 100644 --- a/akka-actors/src/test/scala/ActorSpec.scala +++ b/akka-actors/src/test/scala/ActorSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor import java.util.concurrent.TimeUnit diff --git a/akka-actors/src/test/scala/AllSuite.scala b/akka-actors/src/test/scala/AllSuite.scala index 9609ec8b06..f9bb7c0d1d 100755 --- a/akka-actors/src/test/scala/AllSuite.scala +++ b/akka-actors/src/test/scala/AllSuite.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka import org.scalatest._ diff --git a/akka-actors/src/test/scala/AllTest.scala b/akka-actors/src/test/scala/AllTest.scala index 546f5cc446..991b360639 100644 --- a/akka-actors/src/test/scala/AllTest.scala +++ b/akka-actors/src/test/scala/AllTest.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka import junit.framework.Test import junit.framework.TestCase import junit.framework.TestSuite -import kernel.actor.{ActorSpec, RemoteActorSpec, PersistentActorSpec, InMemoryActorSpec} -import kernel.reactor.{EventBasedSingleThreadDispatcherTest, EventBasedThreadPoolDispatcherTest} -import kernel.util.SchedulerSpec +import actor.{ActorSpec, RemoteActorSpec, PersistentActorSpec, InMemoryActorSpec} +import reactor.{EventBasedSingleThreadDispatcherTest, EventBasedThreadPoolDispatcherTest} +import util.SchedulerSpec object AllTest extends TestCase { def suite(): Test = { diff --git a/akka-actors/src/test/scala/CamelSpec.scala b/akka-actors/src/test/scala/CamelSpec.scala index f988d9daa5..aa64a48cdb 100644 --- a/akka-actors/src/test/scala/CamelSpec.scala +++ b/akka-actors/src/test/scala/CamelSpec.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka -import akka.kernel.config.ActiveObjectGuiceConfigurator +import config.ActiveObjectGuiceConfigurator import annotation.oneway -import kernel.config.ScalaConfig._ +import config.ScalaConfig._ import com.google.inject.{AbstractModule, Scopes} //import com.jteigen.scalatest.JUnit4Runner diff --git a/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala b/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala index 3ac4eee51a..758f9d6cd0 100644 --- a/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala +++ b/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit diff --git a/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala b/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala index c0b205d6f6..a57ad0b825 100644 --- a/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala +++ b/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy import java.util.concurrent.atomic.AtomicBoolean diff --git a/akka-actors/src/test/scala/InMemoryActorSpec.scala b/akka-actors/src/test/scala/InMemoryActorSpec.scala index 5ce06a10b3..eb9b0d4a90 100644 --- a/akka-actors/src/test/scala/InMemoryActorSpec.scala +++ b/akka-actors/src/test/scala/InMemoryActorSpec.scala @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor import junit.framework.TestCase -import kernel.state.TransactionalState +import state.TransactionalState import org.junit.{Test, Before} import org.junit.Assert._ diff --git a/akka-actors/src/test/scala/Messages.scala b/akka-actors/src/test/scala/Messages.scala index 51e735db8c..7e4d5ca66f 100644 --- a/akka-actors/src/test/scala/Messages.scala +++ b/akka-actors/src/test/scala/Messages.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka import akka.serialization.Serializable diff --git a/akka-actors/src/test/scala/RemoteActorSpec.scala b/akka-actors/src/test/scala/RemoteActorSpec.scala index 3387aa8eb0..6e60ebffce 100644 --- a/akka-actors/src/test/scala/RemoteActorSpec.scala +++ b/akka-actors/src/test/scala/RemoteActorSpec.scala @@ -1,8 +1,8 @@ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor import java.util.concurrent.TimeUnit import junit.framework.TestCase -import kernel.nio.{RemoteServer, RemoteClient} +import nio.{RemoteServer, RemoteClient} import org.junit.{Test, Before} import org.junit.Assert._ diff --git a/akka-actors/src/test/scala/RemoteSupervisorSpec.scala b/akka-actors/src/test/scala/RemoteSupervisorSpec.scala index c8e3770cec..2ce554b115 100644 --- a/akka-actors/src/test/scala/RemoteSupervisorSpec.scala +++ b/akka-actors/src/test/scala/RemoteSupervisorSpec.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka import akka.serialization.BinaryString -import kernel.nio.{RemoteClient, RemoteServer} -import kernel.actor.{Supervisor, SupervisorFactory, Actor, StartSupervisor} -import kernel.config.ScalaConfig._ +import nio.{RemoteClient, RemoteServer} +import actor.{Supervisor, SupervisorFactory, Actor, StartSupervisor} +import config.ScalaConfig._ //import com.jteigen.scalatest.JUnit4Runner import org.junit.runner.RunWith diff --git a/akka-actors/src/test/scala/SchedulerSpec.scala b/akka-actors/src/test/scala/SchedulerSpec.scala index bf763046fc..be1e8880b1 100644 --- a/akka-actors/src/test/scala/SchedulerSpec.scala +++ b/akka-actors/src/test/scala/SchedulerSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.kernel.util +package se.scalablesolutions.akka.util import se.scalablesolutions.akka.kernel.actor.Actor diff --git a/akka-actors/src/test/scala/SupervisorSpec.scala b/akka-actors/src/test/scala/SupervisorSpec.scala index d10b987522..2d3ee77ce7 100644 --- a/akka-actors/src/test/scala/SupervisorSpec.scala +++ b/akka-actors/src/test/scala/SupervisorSpec.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka.actor -import kernel.actor.{Supervisor, SupervisorFactory, Actor, StartSupervisor} -import kernel.config.ScalaConfig._ +import actor.{Supervisor, SupervisorFactory, Actor, StartSupervisor} +import config.ScalaConfig._ //import com.jteigen.scalatest.JUnit4Runner import org.junit.runner.RunWith diff --git a/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala b/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala index 1ad5c0b733..a76d02815d 100644 --- a/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala +++ b/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.kernel.reactor +package se.scalablesolutions.akka.reactor import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit diff --git a/akka-actors/src/test/scala/TransactionClasherSpec.scala b/akka-actors/src/test/scala/TransactionClasherSpec.scala index b04f14d120..e9bf2b11a0 100644 --- a/akka-actors/src/test/scala/TransactionClasherSpec.scala +++ b/akka-actors/src/test/scala/TransactionClasherSpec.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.kernel.actor +package se.scalablesolutions.akka.actor import junit.framework.TestCase -import kernel.stm.TransactionRollbackException +import stm.TransactionRollbackException import org.junit.{Test, Before} import org.junit.Assert._ -import kernel.state.TransactionalState +import state.TransactionalState object Log { var log = "" diff --git a/akka-amqp/pom.xml b/akka-amqp/pom.xml index 5e67d7a046..ef45bc8c2f 100644 --- a/akka-amqp/pom.xml +++ b/akka-amqp/pom.xml @@ -16,7 +16,12 @@ - akka-kernel + akka-util + se.scalablesolutions.akka + 0.6 + + + akka-actors se.scalablesolutions.akka 0.6 diff --git a/akka-camel/pom.xml b/akka-camel/pom.xml index 61529e85af..797b8fef53 100644 --- a/akka-camel/pom.xml +++ b/akka-camel/pom.xml @@ -17,6 +17,11 @@ + + akka-util + se.scalablesolutions.akka + 0.6 + akka-actors se.scalablesolutions.akka diff --git a/akka-camel/src/main/scala/ActiveObjectComponent.scala b/akka-camel/src/main/scala/ActiveObjectComponent.scala index 39489156f9..f95fd3c2ed 100644 --- a/akka-camel/src/main/scala/ActiveObjectComponent.scala +++ b/akka-camel/src/main/scala/ActiveObjectComponent.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.camel +package se.scalablesolutions.akka.camel import config.ActiveObjectConfigurator diff --git a/akka-camel/src/main/scala/ActiveObjectConsumer.scala b/akka-camel/src/main/scala/ActiveObjectConsumer.scala index e7be8b0dd9..f9f187de45 100644 --- a/akka-camel/src/main/scala/ActiveObjectConsumer.scala +++ b/akka-camel/src/main/scala/ActiveObjectConsumer.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.camel +package se.scalablesolutions.akka.camel import java.util.concurrent.{BlockingQueue, ExecutorService, Executors, ThreadFactory, TimeUnit} -import kernel.util.Logging +import util.Logging import org.apache.camel.{AsyncCallback, AsyncProcessor, Consumer, Exchange, Processor} import org.apache.camel.impl.ServiceSupport diff --git a/akka-camel/src/main/scala/ActiveObjectEndpoint.scala b/akka-camel/src/main/scala/ActiveObjectEndpoint.scala index 4f9792a2d0..3999c0b897 100644 --- a/akka-camel/src/main/scala/ActiveObjectEndpoint.scala +++ b/akka-camel/src/main/scala/ActiveObjectEndpoint.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.camel +package se.scalablesolutions.akka.camel -import kernel.config.ActiveObjectConfigurator -import kernel.util.Logging +import config.ActiveObjectConfigurator +import util.Logging import java.util.{ArrayList, HashSet, List, Set} import java.util.concurrent.{BlockingQueue, CopyOnWriteArraySet, LinkedBlockingQueue} diff --git a/akka-camel/src/main/scala/ActiveObjectProducer.scala b/akka-camel/src/main/scala/ActiveObjectProducer.scala index f2af00517c..9494510097 100644 --- a/akka-camel/src/main/scala/ActiveObjectProducer.scala +++ b/akka-camel/src/main/scala/ActiveObjectProducer.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.camel +package se.scalablesolutions.akka.camel import java.util.Collection -import kernel.util.Logging; +import util.Logging; import java.util.concurrent.BlockingQueue; import org.apache.camel.{Exchange, AsyncProcessor, AsyncCallback} diff --git a/akka-actors/src/main/scala/config/CamelConfigurator.scala b/akka-camel/src/main/scala/CamelConfigurator.scala similarity index 94% rename from akka-actors/src/main/scala/config/CamelConfigurator.scala rename to akka-camel/src/main/scala/CamelConfigurator.scala index 9cebafed9b..680680f188 100644 --- a/akka-actors/src/main/scala/config/CamelConfigurator.scala +++ b/akka-camel/src/main/scala/CamelConfigurator.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config +package se.scalablesolutions.akka.config import org.apache.camel.{Routes, CamelContext, Endpoint} diff --git a/akka-camel/src/main/scala/MessageDriven.scala b/akka-camel/src/main/scala/MessageDriven.scala index 889ddafeff..3e73a4101b 100644 --- a/akka-camel/src/main/scala/MessageDriven.scala +++ b/akka-camel/src/main/scala/MessageDriven.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.camel +package se.scalablesolutions.akka.camel import org.apache.camel.Exchange diff --git a/akka-camel/src/main/scala/SupervisorAwareCamelContext.scala b/akka-camel/src/main/scala/SupervisorAwareCamelContext.scala index b56d3f8d82..4b9ee8b41d 100644 --- a/akka-camel/src/main/scala/SupervisorAwareCamelContext.scala +++ b/akka-camel/src/main/scala/SupervisorAwareCamelContext.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.camel +package se.scalablesolutions.akka.camel -import kernel.actor.Supervisor -import kernel.util.Logging +import actor.Supervisor +import util.Logging import org.apache.camel.impl.{DefaultCamelContext, DefaultEndpoint, DefaultComponent} /** diff --git a/akka-kernel/pom.xml b/akka-kernel/pom.xml index 4508ab68b6..12929b4a02 100644 --- a/akka-kernel/pom.xml +++ b/akka-kernel/pom.xml @@ -44,7 +44,7 @@ - se.scalablesolutions.akka.kernel.Kernel + se.scalablesolutions.akka.Kernel diff --git a/akka-kernel/src/main/scala/Boot.scala b/akka-kernel/src/main/scala/Boot.scala deleted file mode 100644 index ded1c1c732..0000000000 --- a/akka-kernel/src/main/scala/Boot.scala +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Copyright (C) 2009 Scalable Solutions. - */ - -package se.scalablesolutions.akka - -import java.io.File -import java.net.URLClassLoader -import kernel.util.Logging - -/** - * Bootstraps the Akka server by isolating the server classes and all its dependency JARs into its own classloader. - * - * @author Jonas Bonér - */ -object Boot extends Logging { - - val HOME = { - val home = System.getenv("AKKA_HOME") - if (home == null) throw new IllegalStateException("No 'AKKA_HOME' environment variable set. You have to set 'AKKA_HOME' to the root of the Akka distribution.") - else home - } - val CLASSES = HOME + "/kernel/target/classes" // FIXME remove for dist - val LIB = HOME + "/lib" - val CONFIG = HOME + "/config" - val DEPLOY = HOME + "/deploy" - - /** - * Assumes that the AKKA_HOME directory is set with /bin, /config, /deploy and /lib beneath it holding config files and jars. - * Thus: - * $AKKA_HOME/bin - * $AKKA_HOME/config - * $AKKA_HOME/lib - * $AKKA_HOME/deploy - */ - def main(args: Array[String]): Unit = { - log.info("Bootstrapping Akka server from [AKKA_HOME=%s]", HOME) - - val LIB_DIR = new File(LIB) - val DEPLOY_DIR = new File(DEPLOY) - if (!LIB_DIR.exists) { log.error("Could not find a lib directory with all the akka dependencies at [" + DEPLOY + "]"); System.exit(-1) } - if (!DEPLOY_DIR.exists) { log.error("Could not find a deploy directory at [" + DEPLOY + "]"); System.exit(-1) } - - val toDeploy = for (f <- DEPLOY_DIR.listFiles().toArray.toList.asInstanceOf[List[File]]) yield f.toURL - if (toDeploy.isEmpty) log.warning("No jars could be found in the [" + DEPLOY + "] directory, nothing to deploy") - val libs = for (f <- LIB_DIR.listFiles().toArray.toList.asInstanceOf[List[File]]) yield f.toURL - val urls = new File(CLASSES).toURL :: (libs ::: toDeploy) - - val loader = new URLClassLoader(urls.toArray, ClassLoader.getSystemClassLoader.getParent) - val mainClass = loader.loadClass(args(0)) - val mainMethod = mainClass.getMethod("main", Array(args.getClass): _*) - Thread.currentThread.setContextClassLoader(loader) - - val serverArgs = new Array[String](args.length - 1) - System.arraycopy(args, 1, serverArgs, 0, serverArgs.length) - mainMethod.invoke(null, Array[Object](serverArgs): _*) - } -} - diff --git a/akka-kernel/src/main/scala/Kernel.scala b/akka-kernel/src/main/scala/Kernel.scala index 5c05d16193..43a01b1a7e 100644 --- a/akka-kernel/src/main/scala/Kernel.scala +++ b/akka-kernel/src/main/scala/Kernel.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel +package se.scalablesolutions.akka import com.sun.grizzly.http.SelectorThread import com.sun.grizzly.http.servlet.ServletAdapter @@ -14,34 +14,24 @@ import java.net.URLClassLoader import net.lag.configgy.{Config, Configgy, RuntimeEnvironment, ParseException} -import kernel.rest.AkkaCometServlet -import kernel.nio.RemoteServer -import kernel.state.CassandraStorage -import kernel.util.Logging +import rest.AkkaCometServlet +import nio.RemoteServer +import state.CassandraStorage +import util.Logging /** * @author Jonas Bonér */ object Kernel extends Logging { - val VERSION = "0.6" - val HOME = { - val home = System.getenv("AKKA_HOME") - if (home == null) None - else Some(home) - } - - val config = setupConfig - - val CONFIG_VERSION = config.getString("akka.version", "0") - if (VERSION != CONFIG_VERSION) throw new IllegalStateException("Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]") + import Config._ val BOOT_CLASSES = config.getList("akka.boot") val RUN_REMOTE_SERVICE = config.getBool("akka.remote.service", true) val STORAGE_SYSTEM = config.getString("akka.storage.system", "cassandra") val RUN_REST_SERVICE = config.getBool("akka.rest.service", true) - val REST_HOSTNAME = kernel.Kernel.config.getString("akka.rest.hostname", "localhost") + val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost") val REST_URL = "http://" + REST_HOSTNAME - val REST_PORT = kernel.Kernel.config.getInt("akka.rest.port", 9998) + val REST_PORT = config.getInt("akka.rest.port", 9998) // FIXME add API to shut server down gracefully @volatile private var hasBooted = false @@ -68,27 +58,6 @@ object Kernel extends Logging { } } - def uptime = (System.currentTimeMillis - startTime) / 1000 - - private def setupConfig: Config = { - if (HOME.isDefined) { - try { - val configFile = HOME.get + "/config/akka.conf" - Configgy.configure(configFile) - log.info("AKKA_HOME is defined to [%s], config loaded from [%s].", HOME.get, configFile) - } catch { - case e: ParseException => throw new IllegalStateException("'akka.conf' config file can not be found in [" + HOME + "/config/akka.conf] - aborting. Either add it in the 'config' directory or add it to the classpath.") - } - } else { - try { - Configgy.configureFromResource("akka.conf", getClass.getClassLoader) - log.info("Config loaded from the application classpath.") - } catch { - case e: ParseException => throw new IllegalStateException("'$AKKA_HOME/config/akka.conf' could not be found and no 'akka.conf' can be found on the classpath - aborting. . Either add it in the '$AKKA_HOME/config' directory or add it to the classpath.") - } - } - Configgy.config - } private[akka] def runApplicationBootClasses = { val loader = diff --git a/akka-persistence/pom.xml b/akka-persistence/pom.xml index ab2c31f66b..a0563f1cb0 100644 --- a/akka-persistence/pom.xml +++ b/akka-persistence/pom.xml @@ -16,7 +16,12 @@ - akka-kernel + akka-util + se.scalablesolutions.akka + 0.6 + + + akka-actors se.scalablesolutions.akka 0.6 diff --git a/akka-rest/pom.xml b/akka-rest/pom.xml index b79694aaeb..de7e29e5c8 100644 --- a/akka-rest/pom.xml +++ b/akka-rest/pom.xml @@ -17,6 +17,16 @@ + + akka-util + se.scalablesolutions.akka + 0.6 + + + akka-kernel + se.scalablesolutions.akka + 0.6 + akka-actors se.scalablesolutions.akka diff --git a/akka-rest/src/main/scala/ActorComponentProvider.scala b/akka-rest/src/main/scala/ActorComponentProvider.scala index f7a577f61f..4985bc48de 100755 --- a/akka-rest/src/main/scala/ActorComponentProvider.scala +++ b/akka-rest/src/main/scala/ActorComponentProvider.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.rest +package se.scalablesolutions.akka.rest import com.sun.jersey.core.spi.component.ioc.IoCFullyManagedComponentProvider -import kernel.config.Configurator -import kernel.util.Logging +import config.Configurator +import util.Logging class ActorComponentProvider(val clazz: Class[_], val configurators: List[Configurator]) extends IoCFullyManagedComponentProvider with Logging { diff --git a/akka-rest/src/main/scala/ActorComponentProviderFactory.scala b/akka-rest/src/main/scala/ActorComponentProviderFactory.scala index 1a46e44ff8..5917ea2ea8 100755 --- a/akka-rest/src/main/scala/ActorComponentProviderFactory.scala +++ b/akka-rest/src/main/scala/ActorComponentProviderFactory.scala @@ -2,17 +2,15 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.rest +package se.scalablesolutions.akka.rest -import kernel.Kernel -import util.Logging import javax.ws.rs.core.Context import com.sun.jersey.core.spi.component.ioc.{IoCComponentProvider,IoCComponentProviderFactory} import com.sun.jersey.core.spi.component.{ComponentContext} import config.Configurator - +import util.Logging class ActorComponentProviderFactory(val configurators: List[Configurator]) extends IoCComponentProviderFactory with Logging { diff --git a/akka-rest/src/main/scala/AkkaServlet.scala b/akka-rest/src/main/scala/AkkaServlet.scala index 885d3e6321..baff100abc 100755 --- a/akka-rest/src/main/scala/AkkaServlet.scala +++ b/akka-rest/src/main/scala/AkkaServlet.scala @@ -2,9 +2,8 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.rest +package se.scalablesolutions.akka.rest -import kernel.Kernel import config.ConfiguratorRepository import util.Logging @@ -32,7 +31,7 @@ import scala.collection.jcl.Conversions._ class AkkaServlet extends ServletContainer with AtmosphereServletProcessor with Logging { override def initiate(rc: ResourceConfig, wa: WebApplication) = { - Kernel.boot // will boot if not already booted by 'main' + akka.Kernel.boot // will boot if not already booted by 'main' val configurators = ConfiguratorRepository.getConfiguratorsFor(getServletContext) rc.getClasses.addAll(configurators.flatMap(_.getComponentInterfaces)) diff --git a/akka-rest/src/main/scala/NodeWriter.scala b/akka-rest/src/main/scala/NodeWriter.scala index c301d9c2b5..58c127b411 100755 --- a/akka-rest/src/main/scala/NodeWriter.scala +++ b/akka-rest/src/main/scala/NodeWriter.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.rest +package se.scalablesolutions.akka.rest import java.io.OutputStream import java.lang.annotation.Annotation diff --git a/akka-samples-java/pom.xml b/akka-samples-java/pom.xml index ec1c023ae9..2a7cf2adff 100644 --- a/akka-samples-java/pom.xml +++ b/akka-samples-java/pom.xml @@ -16,12 +16,32 @@ - akka-kernel + akka-util-java se.scalablesolutions.akka 0.6 - akka-util-java + akka-util + se.scalablesolutions.akka + 0.6 + + + akka-actors + se.scalablesolutions.akka + 0.6 + + + akka-persistence + se.scalablesolutions.akka + 0.6 + + + akka-rest + se.scalablesolutions.akka + 0.6 + + + akka-kernel se.scalablesolutions.akka 0.6 diff --git a/akka-samples-lift/pom.xml b/akka-samples-lift/pom.xml index ab7b8f7554..c48bed46e0 100644 --- a/akka-samples-lift/pom.xml +++ b/akka-samples-lift/pom.xml @@ -19,14 +19,34 @@ - + + akka-util-java se.scalablesolutions.akka - akka-kernel 0.6 + akka-util + se.scalablesolutions.akka + 0.6 + + + akka-actors + se.scalablesolutions.akka + 0.6 + + + akka-persistence + se.scalablesolutions.akka + 0.6 + + + akka-rest + se.scalablesolutions.akka + 0.6 + + + akka-kernel se.scalablesolutions.akka - akka-util-java 0.6 diff --git a/akka-samples-scala/pom.xml b/akka-samples-scala/pom.xml index 6c2968f970..59e9948e3d 100644 --- a/akka-samples-scala/pom.xml +++ b/akka-samples-scala/pom.xml @@ -15,13 +15,33 @@ - - akka-kernel + + akka-util-java se.scalablesolutions.akka 0.6 - akka-util-java + akka-util + se.scalablesolutions.akka + 0.6 + + + akka-actors + se.scalablesolutions.akka + 0.6 + + + akka-persistence + se.scalablesolutions.akka + 0.6 + + + akka-rest + se.scalablesolutions.akka + 0.6 + + + akka-kernel se.scalablesolutions.akka 0.6 diff --git a/akka-stm/pom.xml b/akka-stm/pom.xml deleted file mode 100644 index 12da6ab9d0..0000000000 --- a/akka-stm/pom.xml +++ /dev/null @@ -1,95 +0,0 @@ - - 4.0.0 - - akka-kernel - Akka Kernel Module - - jar - - - akka - se.scalablesolutions.akka - 0.6 - ../pom.xml - - - - - - akka-actors - se.scalablesolutions.akka - 0.6 - - - org.apache - zookeeper - 3.1.0 - - - - - - - maven-assembly-plugin - 2.2-beta-2 - - - create-executable-jar - install - - single - - - - - jar-with-dependencies - - - - - se.scalablesolutions.akka.kernel.Kernel - - - - - - - - maven-antrun-plugin - - - install - - - - - - - run - - - - - - - - false - ../config - - akka.conf - akka-reference.conf - - - - false - src/main/resources - - META-INF/* - - - - - diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/config/ActiveObjectGuiceModule.java b/akka-util-java/src/main/java/se/scalablesolutions/akka/config/ActiveObjectGuiceModule.java similarity index 94% rename from akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/config/ActiveObjectGuiceModule.java rename to akka-util-java/src/main/java/se/scalablesolutions/akka/config/ActiveObjectGuiceModule.java index 052409d5fd..6ec4dae990 100644 --- a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/config/ActiveObjectGuiceModule.java +++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/config/ActiveObjectGuiceModule.java @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config; +package se.scalablesolutions.akka.config; import java.util.List; diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/config/DependencyBinding.java b/akka-util-java/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java similarity index 90% rename from akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/config/DependencyBinding.java rename to akka-util-java/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java index 40d45869a8..feda1c8486 100644 --- a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/config/DependencyBinding.java +++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.config; +package se.scalablesolutions.akka.config; /** * @author Jonas Bonér diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.java b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java similarity index 79% rename from akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.java rename to akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java index c36d54c2c7..38022db312 100644 --- a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.java +++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java @@ -1,6 +1,6 @@ // Generated by the protocol buffer compiler. DO NOT EDIT! -package se.scalablesolutions.akka.kernel.nio.protobuf; +package se.scalablesolutions.akka.nio.protobuf; public final class RemoteProtocol { private RemoteProtocol() {} @@ -23,13 +23,13 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_descriptor; + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_descriptor; } @Override protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_fieldAccessorTable; + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_fieldAccessorTable; } // required uint64 id = 1; @@ -217,57 +217,57 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistry extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom(byte[] data) + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( byte[] data, com.google.protobuf.ExtensionRegistry extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom(java.io.InputStream input) + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistry extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseDelimitedFrom(java.io.InputStream input) + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeDelimitedFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseDelimitedFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistry extensionRegistry) throws java.io.IOException { return newBuilder().mergeDelimitedFrom(input, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistry extensionRegistry) throws java.io.IOException { @@ -277,26 +277,26 @@ public final class RemoteProtocol { public static Builder newBuilder() { return new Builder(); } public Builder newBuilderForType() { return new Builder(); } - public static Builder newBuilder(se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest prototype) { + public static Builder newBuilder(se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest prototype) { return new Builder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { - // Construct using se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest.newBuilder() + // Construct using se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.newBuilder() private Builder() {} - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest result = new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest(); + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest result = new se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest(); @Override - protected se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest internalGetResult() { + protected se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest internalGetResult() { return result; } @Override public Builder clear() { - result = new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest(); + result = new se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest(); return this; } @@ -308,14 +308,14 @@ public final class RemoteProtocol { @Override public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest.getDescriptor(); + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.getDescriptor(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest getDefaultInstanceForType() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest.getDefaultInstance(); + public se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest getDefaultInstanceForType() { + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.getDefaultInstance(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest build() { + public se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest build() { if (result != null && !isInitialized()) { throw new com.google.protobuf.UninitializedMessageException( result); @@ -323,7 +323,7 @@ public final class RemoteProtocol { return buildPartial(); } - private se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest buildParsed() + private se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw new com.google.protobuf.UninitializedMessageException( @@ -332,27 +332,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest buildPartial() { + public se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest returnMe = result; + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest returnMe = result; result = null; return returnMe; } @Override public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest) { - return mergeFrom((se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest)other); + if (other instanceof se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest) { + return mergeFrom((se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest other) { - if (other == se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest.getDefaultInstance()) return this; + public Builder mergeFrom(se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest other) { + if (other == se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.getDefaultInstance()) return this; if (other.hasId()) { setId(other.getId()); } @@ -684,7 +684,7 @@ public final class RemoteProtocol { } static { - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.getDescriptor(); + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.getDescriptor(); } } @@ -704,13 +704,13 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_descriptor; + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_descriptor; } @Override protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_fieldAccessorTable; + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_fieldAccessorTable; } // required uint64 id = 1; @@ -851,57 +851,57 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistry extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom(byte[] data) + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( byte[] data, com.google.protobuf.ExtensionRegistry extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom(java.io.InputStream input) + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistry extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseDelimitedFrom(java.io.InputStream input) + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeDelimitedFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseDelimitedFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistry extensionRegistry) throws java.io.IOException { return newBuilder().mergeDelimitedFrom(input, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( + public static se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistry extensionRegistry) throws java.io.IOException { @@ -909,28 +909,28 @@ public final class RemoteProtocol { .buildParsed(); } - public static se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder newBuilder() { return new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder newBuilderForType() { return new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder(); } - public static Builder newBuilder(se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply prototype) { - return new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder().mergeFrom(prototype); + public static Builder newBuilder() { return new Builder(); } + public Builder newBuilderForType() { return new Builder(); } + public static Builder newBuilder(se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply prototype) { + return new Builder().mergeFrom(prototype); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder toBuilder() { return newBuilder(this); } + public Builder toBuilder() { return newBuilder(this); } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { - // Construct using se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.newBuilder() + // Construct using se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply.newBuilder() private Builder() {} - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply result = new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply(); + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply result = new se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply(); @Override - protected se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply internalGetResult() { + protected se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply internalGetResult() { return result; } @Override public Builder clear() { - result = new se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply(); + result = new se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply(); return this; } @@ -942,14 +942,14 @@ public final class RemoteProtocol { @Override public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.getDescriptor(); + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply.getDescriptor(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply getDefaultInstanceForType() { - return se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.getDefaultInstance(); + public se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply getDefaultInstanceForType() { + return se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply.getDefaultInstance(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply build() { + public se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply build() { if (result != null && !isInitialized()) { throw new com.google.protobuf.UninitializedMessageException( result); @@ -957,7 +957,7 @@ public final class RemoteProtocol { return buildPartial(); } - private se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply buildParsed() + private se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw new com.google.protobuf.UninitializedMessageException( @@ -966,27 +966,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply buildPartial() { + public se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply returnMe = result; + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply returnMe = result; result = null; return returnMe; } @Override public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply) { - return mergeFrom((se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply)other); + if (other instanceof se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply) { + return mergeFrom((se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply other) { - if (other == se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.getDefaultInstance()) return this; + public Builder mergeFrom(se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply other) { + if (other == se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply.getDefaultInstance()) return this; if (other.hasId()) { setId(other.getId()); } @@ -1240,20 +1240,20 @@ public final class RemoteProtocol { } static { - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.getDescriptor(); + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.getDescriptor(); } } private static com.google.protobuf.Descriptors.Descriptor - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_descriptor; + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_fieldAccessorTable; + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_descriptor; + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_fieldAccessorTable; + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -1263,40 +1263,40 @@ public final class RemoteProtocol { descriptor; static { java.lang.String descriptorData = - "\nBse/scalablesolutions/akka/kernel/nio/p" + - "rotobuf/RemoteProtocol.proto\022-se.scalabl" + - "esolutions.akka.kernel.nio.protobuf\"\326\001\n\r" + - "RemoteRequest\022\n\n\002id\030\001 \002(\004\022\020\n\010protocol\030\002 " + - "\002(\r\022\017\n\007message\030\003 \002(\014\022\027\n\017messageManifest\030" + - "\004 \001(\014\022\016\n\006method\030\005 \001(\t\022\016\n\006target\030\006 \002(\t\022\017\n" + - "\007timeout\030\007 \002(\004\022\026\n\016supervisorUuid\030\010 \001(\t\022\017" + - "\n\007isActor\030\t \002(\010\022\020\n\010isOneWay\030\n \002(\010\022\021\n\tisE" + - "scaped\030\013 \002(\010\"\247\001\n\013RemoteReply\022\n\n\002id\030\001 \002(\004" + - "\022\020\n\010protocol\030\002 \001(\r\022\017\n\007message\030\003 \001(\014\022\027\n\017m" + - "essageManifest\030\004 \001(\014\022\021\n\texception\030\005 \001(\t\022" + - "\026\n\016supervisorUuid\030\006 \001(\t\022\017\n\007isActor\030\007 \002(\010" + - "\022\024\n\014isSuccessful\030\010 \002(\010B\002H\001"; + "\n;se/scalablesolutions/akka/nio/protobuf" + + "/RemoteProtocol.proto\022&se.scalablesoluti" + + "ons.akka.nio.protobuf\"\326\001\n\rRemoteRequest\022" + + "\n\n\002id\030\001 \002(\004\022\020\n\010protocol\030\002 \002(\r\022\017\n\007message" + + "\030\003 \002(\014\022\027\n\017messageManifest\030\004 \001(\014\022\016\n\006metho" + + "d\030\005 \001(\t\022\016\n\006target\030\006 \002(\t\022\017\n\007timeout\030\007 \002(\004" + + "\022\026\n\016supervisorUuid\030\010 \001(\t\022\017\n\007isActor\030\t \002(" + + "\010\022\020\n\010isOneWay\030\n \002(\010\022\021\n\tisEscaped\030\013 \002(\010\"\247" + + "\001\n\013RemoteReply\022\n\n\002id\030\001 \002(\004\022\020\n\010protocol\030\002" + + " \001(\r\022\017\n\007message\030\003 \001(\014\022\027\n\017messageManifest" + + "\030\004 \001(\014\022\021\n\texception\030\005 \001(\t\022\026\n\016supervisorU" + + "uid\030\006 \001(\t\022\017\n\007isActor\030\007 \002(\010\022\024\n\014isSuccessf" + + "ul\030\010 \002(\010B\002H\001"; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { public com.google.protobuf.ExtensionRegistry assignDescriptors( com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_descriptor = + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_descriptor = getDescriptor().getMessageTypes().get(0); - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_fieldAccessorTable = new + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteRequest_descriptor, + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_descriptor, new java.lang.String[] { "Id", "Protocol", "Message", "MessageManifest", "Method", "Target", "Timeout", "SupervisorUuid", "IsActor", "IsOneWay", "IsEscaped", }, - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest.class, - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteRequest.Builder.class); - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_descriptor = + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.class, + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.Builder.class); + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_descriptor = getDescriptor().getMessageTypes().get(1); - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_fieldAccessorTable = new + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_se_scalablesolutions_akka_kernel_nio_protobuf_RemoteReply_descriptor, + internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_descriptor, new java.lang.String[] { "Id", "Protocol", "Message", "MessageManifest", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", }, - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.class, - se.scalablesolutions.akka.kernel.nio.protobuf.RemoteProtocol.RemoteReply.Builder.class); + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply.class, + se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteReply.Builder.class); return null; } }; diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.proto b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto similarity index 81% rename from akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.proto rename to akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto index b58fb392b3..148b3960ae 100644 --- a/akka-util-java/src/main/java/se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.proto +++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto @@ -2,12 +2,12 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.nio.protobuf; +package se.scalablesolutions.akka.nio.protobuf; /* Compile with: - cd ./util-java/src/main/java - protoc se/scalablesolutions/akka/kernel/nio/protobuf/RemoteProtocol.proto --java_out . + cd ./akka-util-java/src/main/java + protoc se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto --java_out . */ option optimize_for = SPEED; diff --git a/akka-util/pom.xml b/akka-util/pom.xml new file mode 100644 index 0000000000..b8645bd74d --- /dev/null +++ b/akka-util/pom.xml @@ -0,0 +1,30 @@ + + 4.0.0 + + akka-util + Akka Util Module + + jar + + + akka + se.scalablesolutions.akka + 0.6 + ../pom.xml + + + + + org.scala-lang + scala-library + 2.7.5 + + + net.lag + configgy + 1.3 + + + + diff --git a/akka-util/src/main/scala/Config.scala b/akka-util/src/main/scala/Config.scala new file mode 100644 index 0000000000..4894a8de8e --- /dev/null +++ b/akka-util/src/main/scala/Config.scala @@ -0,0 +1,47 @@ +/** + * Copyright (C) 2009 Scalable Solutions. + */ + +package se.scalablesolutions.akka + +import util.Logging + +import net.lag.configgy.{Config => ConfiggyConfig, Configgy, RuntimeEnvironment, ParseException} + +/** + * @author Jonas Bonér + */ +object Config extends Logging { + val VERSION = "0.6" + val HOME = { + val home = System.getenv("AKKA_HOME") + if (home == null) None + else Some(home) + } + + val config = { + if (HOME.isDefined) { + try { + val configFile = HOME.get + "/config/akka.conf" + Configgy.configure(configFile) + log.info("AKKA_HOME is defined to [%s], config loaded from [%s].", HOME.get, configFile) + } catch { + case e: ParseException => throw new IllegalStateException("'akka.conf' config file can not be found in [" + HOME + "/config/akka.conf] - aborting. Either add it in the 'config' directory or add it to the classpath.") + } + } else { + try { + Configgy.configureFromResource("akka.conf", getClass.getClassLoader) + log.info("Config loaded from the application classpath.") + } catch { + case e: ParseException => throw new IllegalStateException("'$AKKA_HOME/config/akka.conf' could not be found and no 'akka.conf' can be found on the classpath - aborting. . Either add it in the '$AKKA_HOME/config' directory or add it to the classpath.") + } + } + Configgy.config + } + + val CONFIG_VERSION = config.getString("akka.version", "0") + if (VERSION != CONFIG_VERSION) throw new IllegalStateException("Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]") + val startTime = System.currentTimeMillis + + def uptime = (System.currentTimeMillis - startTime) / 1000 +} diff --git a/akka-actors/src/main/scala/util/HashCode.scala b/akka-util/src/main/scala/HashCode.scala similarity index 97% rename from akka-actors/src/main/scala/util/HashCode.scala rename to akka-util/src/main/scala/HashCode.scala index 7d71376172..0ce21f6a92 100755 --- a/akka-actors/src/main/scala/util/HashCode.scala +++ b/akka-util/src/main/scala/HashCode.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.util +package se.scalablesolutions.akka.util import java.lang.reflect.{Array => JArray} import java.lang.{Float => JFloat, Double => JDouble} diff --git a/akka-actors/src/main/scala/util/Helpers.scala b/akka-util/src/main/scala/Helpers.scala similarity index 97% rename from akka-actors/src/main/scala/util/Helpers.scala rename to akka-util/src/main/scala/Helpers.scala index 6d43f06030..e0840c6b62 100644 --- a/akka-actors/src/main/scala/util/Helpers.scala +++ b/akka-util/src/main/scala/Helpers.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.util +package se.scalablesolutions.akka.util import java.security.MessageDigest import java.util.concurrent.locks.ReentrantReadWriteLock diff --git a/akka-actors/src/main/scala/util/Logging.scala b/akka-util/src/main/scala/Logging.scala similarity index 97% rename from akka-actors/src/main/scala/util/Logging.scala rename to akka-util/src/main/scala/Logging.scala index 655dfce5db..39f2afee21 100644 --- a/akka-actors/src/main/scala/util/Logging.scala +++ b/akka-util/src/main/scala/Logging.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009 Scalable Solutions. */ -package se.scalablesolutions.akka.kernel.util +package se.scalablesolutions.akka.util import java.util.logging.Level import net.lag.configgy.Config diff --git a/akka.iml b/akka.iml deleted file mode 100644 index f748697a86..0000000000 --- a/akka.iml +++ /dev/null @@ -1,18 +0,0 @@ - - - - - - - - - - - - - - - - - - diff --git a/akka.ipr b/akka.ipr deleted file mode 100644 index 068b33fd4c..0000000000 --- a/akka.ipr +++ /dev/null @@ -1,1715 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/akka.iws b/akka.iws deleted file mode 100644 index 1d9fcc5852..0000000000 --- a/akka.iws +++ /dev/null @@ -1,559 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - localhost - 5050 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/pom.xml b/pom.xml index c6f237274d..0fa7b50324 100644 --- a/pom.xml +++ b/pom.xml @@ -4,7 +4,7 @@ 4.0.0 - Akka Actor Kernel + Akka Transactors akka se.scalablesolutions.akka 0.6 @@ -19,12 +19,13 @@ akka-util-java + akka-util akka-actors - akka-stm akka-persistence akka-rest akka-camel akka-amqp + akka-kernel akka-fun-test-java akka-samples-scala akka-samples-lift