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
-
-
-
-
-
-
-
-
-
-
-
-
-
-
- $PROJECT_DIR$/../../../.subversion
- false
- 125
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
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