diff --git a/akka-actor/src/main/scala/akka/config/Config.scala b/akka-actor/src/main/scala/akka/config/Config.scala
index 0571449b67..371e61b640 100644
--- a/akka-actor/src/main/scala/akka/config/Config.scala
+++ b/akka-actor/src/main/scala/akka/config/Config.scala
@@ -69,7 +69,7 @@ object Config {
"\n\tdue to: " + e.toString)
}
Configgy.config
- } else if (HOME.isDefined) {
+ } else if (HOME.isDefined) {
try {
val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/" + confName
Configgy.configure(configFile)
diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
index 4d33bf03ce..bb58c8600a 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
@@ -5,9 +5,9 @@
package akka.dispatch
import akka.actor.{Actor, ActorRef}
+import akka.actor.newUuid
import akka.config.Config._
import akka.util.{Duration, Logging}
-import akka.actor.newUuid
import net.lag.configgy.ConfigMap
diff --git a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala
index ee5dd890b8..2673d0af33 100644
--- a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala
@@ -5,12 +5,11 @@
package akka.dispatch
import akka.actor.{ActorRef, IllegalActorStateException}
-import akka.util.ReflectiveAccess.AkkaCloudModule
+import akka.util.{ReflectiveAccess, Switch}
import java.util.Queue
-import akka.util.Switch
import java.util.concurrent.atomic.AtomicReference
-import java.util.concurrent. {ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue}
+import java.util.concurrent.{ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue}
/**
* Default settings are:
@@ -119,17 +118,10 @@ class ExecutorBasedEventDrivenDispatcher(
}
/**
- * Creates and returns a durable mailbox for the given actor.
+ * Creates and returns a durable mailbox for the given actor.
*/
- def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = mailboxType match {
- // FIXME make generic (work for TypedActor as well)
- case FileBasedDurableMailbox(serializer) => AkkaCloudModule.createFileBasedMailbox(actorRef).asInstanceOf[MessageQueue]
- case ZooKeeperBasedDurableMailbox(serializer) => AkkaCloudModule.createZooKeeperBasedMailbox(actorRef).asInstanceOf[MessageQueue]
- case BeanstalkBasedDurableMailbox(serializer) => AkkaCloudModule.createBeanstalkBasedMailbox(actorRef).asInstanceOf[MessageQueue]
- case RedisBasedDurableMailbox(serializer) => AkkaCloudModule.createRedisBasedMailbox(actorRef).asInstanceOf[MessageQueue]
- case AMQPBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("AMQPBasedDurableMailbox is not yet supported")
- case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported")
- }
+ private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef =
+ createMailbox(mailboxType.mailboxImplClassname, actorRef)
private[akka] def start = log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput)
diff --git a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
index 69aa5d9365..42f3e0f4cf 100644
--- a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
@@ -4,13 +4,15 @@
package akka.dispatch
-import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque}
import akka.actor.{Actor, ActorRef, IllegalActorStateException}
import akka.util.Switch
+
import java.util.concurrent. {ExecutorService, CopyOnWriteArrayList}
import java.util.concurrent.atomic.AtomicReference
+import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque}
+
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
* that all actors using the same instance of this dispatcher can process all messages that have been sent to one of the actors. I.e. the
@@ -222,15 +224,8 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
/**
* Creates and returns a durable mailbox for the given actor.
*/
- private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = mailboxType match {
- // FIXME make generic (work for TypedActor as well)
- case FileBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("FileBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
- case ZooKeeperBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("ZooKeeperBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
- case BeanstalkBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("BeanstalkBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
- case RedisBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("RedisBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
- case AMQPBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("AMQPBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
- case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
- }
+ private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef =
+ createMailbox(mailboxType.mailboxImplClassname, actorRef)
private[akka] override def register(actorRef: ActorRef) = {
verifyActorsAreOfSameType(actorRef)
diff --git a/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala
index 1031ae4c9a..d35fc46617 100644
--- a/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala
@@ -5,6 +5,7 @@
package akka.dispatch
import akka.actor.ActorRef
+import akka.util.Switch
import org.fusesource.hawtdispatch.DispatchQueue
import org.fusesource.hawtdispatch.ScalaDispatch._
@@ -13,7 +14,6 @@ import org.fusesource.hawtdispatch.ListEventAggregator
import java.util.concurrent.atomic.{AtomicInteger, AtomicBoolean}
import java.util.concurrent.CountDownLatch
-import akka.util.Switch
/**
* Holds helper methods for working with actors that are using a HawtDispatcher as it's dispatcher.
diff --git a/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala
index 7e81d4a598..d6c13bfda4 100644
--- a/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala
@@ -5,7 +5,6 @@
package akka.dispatch
import akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException}
-import akka.util.ReflectiveAccess.AkkaCloudModule
import akka.AkkaException
import java.util.{Queue, List}
@@ -42,15 +41,7 @@ case class BoundedMailbox(
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
}
-abstract class DurableMailboxType(val serializer: AkkaCloudModule.Serializer) extends MailboxType {
- if (serializer eq null) throw new IllegalArgumentException("The serializer for DurableMailboxType can not be null")
-}
-case class FileBasedDurableMailbox(ser: AkkaCloudModule.Serializer) extends DurableMailboxType(ser)
-case class RedisBasedDurableMailbox(ser: AkkaCloudModule.Serializer) extends DurableMailboxType(ser)
-case class BeanstalkBasedDurableMailbox(ser: AkkaCloudModule.Serializer) extends DurableMailboxType(ser)
-case class ZooKeeperBasedDurableMailbox(ser: AkkaCloudModule.Serializer) extends DurableMailboxType(ser)
-case class AMQPBasedDurableMailbox(ser: AkkaCloudModule.Serializer) extends DurableMailboxType(ser)
-case class JMSBasedDurableMailbox(ser: AkkaCloudModule.Serializer) extends DurableMailboxType(ser)
+case class DurableMailboxType(mailboxImplClassname: String) extends MailboxType
class DefaultUnboundedMessageQueue(blockDequeue: Boolean)
extends LinkedBlockingQueue[MessageInvocation] with MessageQueue {
diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
index 467bccd13e..b766a7a974 100644
--- a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
@@ -6,7 +6,8 @@ package akka.dispatch
import java.util.concurrent._
import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong}
-import akka.util. {Switch, ReentrantGuard, Logging, HashCode}
+
+import akka.util.{Switch, ReentrantGuard, Logging, HashCode, ReflectiveAccess}
import akka.actor._
/**
@@ -59,11 +60,27 @@ object MessageDispatcher {
*/
trait MessageDispatcher extends MailboxFactory with Logging {
import MessageDispatcher._
+
protected val uuids = new ConcurrentSkipListSet[Uuid]
protected val guard = new ReentrantGuard
- private var shutdownSchedule = UNSCHEDULED //This can be non-volatile since it is protected by guard withGuard
protected val active = new Switch(false)
+ private var shutdownSchedule = UNSCHEDULED //This can be non-volatile since it is protected by guard withGuard
+
+ /**
+ * Creates and returns a mailbox for the given actor.
+ */
+ def createMailbox(mailboxImplClassname: String, actorRef: ActorRef): MessageQueue = {
+ ReflectiveAccess.createInstance(
+ mailboxImplClassname,
+ Array(classOf[ActorRef]),
+ Array(actorRef).asInstanceOf[Array[AnyRef]],
+ ReflectiveAccess.loader)
+ .getOrElse(throw new IllegalActorStateException(
+ "Could not create mailbox [" + mailboxImplClassname + "] for actor [" + actorRef + "]"))
+ .asInstanceOf[MessageQueue]
+ }
+
/**
* Attaches the specified actorRef to this dispatcher
*/
diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
index b5a4c7edb7..f359ab9b98 100644
--- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
+++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
@@ -22,11 +22,9 @@ object ReflectiveAccess extends Logging {
lazy val isRemotingEnabled = RemoteClientModule.isEnabled
lazy val isTypedActorEnabled = TypedActorModule.isEnabled
- lazy val isAkkaCloudEnabled = AkkaCloudModule.isEnabled
def ensureRemotingEnabled = RemoteClientModule.ensureEnabled
def ensureTypedActorEnabled = TypedActorModule.ensureEnabled
- def ensureAkkaCloudEnabled = AkkaCloudModule.ensureEnabled
/**
* Reflective access to the RemoteClient module.
@@ -173,49 +171,6 @@ object ReflectiveAccess extends Logging {
}
}
- object AkkaCloudModule {
-
- type Mailbox = {
- def enqueue(message: MessageInvocation)
- def dequeue: MessageInvocation
- }
-
- type Serializer = {
- def toBinary(obj: AnyRef): Array[Byte]
- def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef
- }
-
- lazy val isEnabled = clusterObjectInstance.isDefined
-
- val clusterObjectInstance: Option[AnyRef] =
- getObjectFor("akka.cloud.cluster.Cluster$")
-
- val serializerClass: Option[Class[_]] =
- getClassFor("akka.serialization.Serializer")
-
- def ensureEnabled = if (!isEnabled) throw new ModuleNotAvailableException(
- "Feature is only available in Akka Cloud")
-
- def createFileBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.cloud.cluster.FileBasedMailbox", actorRef)
-
- def createZooKeeperBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.cloud.cluster.ZooKeeperBasedMailbox", actorRef)
-
- def createBeanstalkBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.cloud.cluster.BeanstalkBasedMailbox", actorRef)
-
- def createRedisBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.cloud.cluster.RedisBasedMailbox", actorRef)
-
- private def createMailbox(mailboxClassname: String, actorRef: ActorRef): Mailbox = {
- ensureEnabled
- createInstance(
- mailboxClassname,
- Array(classOf[ActorRef]),
- Array(actorRef).asInstanceOf[Array[AnyRef]],
- loader)
- .getOrElse(throw new IllegalActorStateException("Could not create durable mailbox [" + mailboxClassname + "] for actor [" + actorRef + "]"))
- .asInstanceOf[Mailbox]
- }
- }
-
val noParams = Array[Class[_]]()
val noArgs = Array[AnyRef]()
diff --git a/akka-camel/src/main/scala/akka/CamelService.scala b/akka-camel/src/main/scala/akka/CamelService.scala
index b546636610..da71701ae1 100644
--- a/akka-camel/src/main/scala/akka/CamelService.scala
+++ b/akka-camel/src/main/scala/akka/CamelService.scala
@@ -26,7 +26,7 @@ trait CamelService extends Bootable with Logging {
private[camel] val consumerPublisher = actorOf[ConsumerPublisher]
private[camel] val publishRequestor = actorOf[PublishRequestor]
- private val serviceEnabled = config.getBool("akka.camel.service", true)
+ private val serviceEnabled = config.getList("akka.enabled-modules").exists(_ == "camel")
/**
* Starts this CamelService unless akka.camel.service is set to false.
diff --git a/akka-http/src/main/scala/EmbeddedAppServer.scala b/akka-http/src/main/scala/EmbeddedAppServer.scala
index 1b6f31e479..d46cb775c5 100644
--- a/akka-http/src/main/scala/EmbeddedAppServer.scala
+++ b/akka-http/src/main/scala/EmbeddedAppServer.scala
@@ -21,23 +21,25 @@ import org.eclipse.jetty.server.handler.{HandlerList, HandlerCollection, Context
* Handles the Akka Comet Support (load/unload)
*/
trait EmbeddedAppServer extends Bootable with Logging {
- self : BootableActorLoaderService =>
+ self: BootableActorLoaderService =>
import akka.config.Config._
- val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost")
- val REST_PORT = config.getInt("akka.rest.port", 9998)
+ val REST_HOSTNAME = config.getString("akka.http.hostname", "localhost")
+ val REST_PORT = config.getInt("akka.http.port", 9998)
+
+ val isRestEnabled = config.getList("akka.enabled-modules").exists(_ == "http")
protected var server: Option[Server] = None
abstract override def onLoad = {
super.onLoad
- if (config.getBool("akka.rest.service", true)) {
- log.info("Attempting to start Akka REST service (Jersey)")
+ if (isRestEnabled) {
+ log.info("Attempting to start Akka HTTP service")
- System.setProperty("jetty.port",REST_PORT.toString)
- System.setProperty("jetty.host",REST_HOSTNAME)
- System.setProperty("jetty.home",HOME.getOrElse(throwNoAkkaHomeException) + "/deploy/root")
+ System.setProperty("jetty.port", REST_PORT.toString)
+ System.setProperty("jetty.host", REST_HOSTNAME)
+ System.setProperty("jetty.home", HOME.getOrElse(throwNoAkkaHomeException) + "/deploy/root")
val configuration = new XmlConfiguration(
new File(HOME.getOrElse(throwNoAkkaHomeException) + "/config/microkernel-server.xml").toURI.toURL)
@@ -58,16 +60,15 @@ trait EmbeddedAppServer extends Bootable with Logging {
s.start()
s
}
- log.info("Akka REST service started (Jersey)")
+ log.info("Akka HTTP service started")
}
}
abstract override def onUnload = {
super.onUnload
- server foreach { t => {
+ server foreach { t =>
log.info("Shutting down REST service (Jersey)")
t.stop()
- }
}
}
}
diff --git a/akka-http/src/main/scala/ListWriter.scala b/akka-http/src/main/scala/ListWriter.scala
index 3a2c69d02a..b2addd6cd7 100644
--- a/akka-http/src/main/scala/ListWriter.scala
+++ b/akka-http/src/main/scala/ListWriter.scala
@@ -1,10 +1,11 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB
*/
-package akka.rest
+package akka.http
+
+import akka.serialization.Serializer
import java.io.OutputStream
-import akka.serialization.Serializer
import javax.ws.rs.core.{MultivaluedMap, MediaType}
import javax.ws.rs.ext.{MessageBodyWriter, Provider}
import javax.ws.rs.Produces
diff --git a/akka-http/src/main/scala/Security.scala b/akka-http/src/main/scala/Security.scala
index b6aec9a1f0..e9152970fa 100644
--- a/akka-http/src/main/scala/Security.scala
+++ b/akka-http/src/main/scala/Security.scala
@@ -101,8 +101,8 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
}
lazy val authenticatorFQN = {
- val auth = Config.config.getString("akka.rest.authenticator", "N/A")
- if (auth == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.authenticator' is not defined in 'akka.conf'")
+ val auth = Config.config.getString("akka.http.authenticator", "N/A")
+ if (auth == "N/A") throw new IllegalActorStateException("The config option 'akka.http.authenticator' is not defined in 'akka.conf'")
auth
}
@@ -399,8 +399,8 @@ trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] w
* principal name for the HTTP kerberos service, i.e HTTP/ { server } @ { realm }
*/
lazy val servicePrincipal = {
- val p = Config.config.getString("akka.rest.kerberos.servicePrincipal", "N/A")
- if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.kerberos.servicePrincipal' is not defined in 'akka.conf'")
+ val p = Config.config.getString("akka.http.kerberos.servicePrincipal", "N/A")
+ if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.http.kerberos.servicePrincipal' is not defined in 'akka.conf'")
p
}
@@ -408,21 +408,21 @@ trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] w
* keytab location with credentials for the service principal
*/
lazy val keyTabLocation = {
- val p = Config.config.getString("akka.rest.kerberos.keyTabLocation", "N/A")
- if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.kerberos.keyTabLocation' is not defined in 'akka.conf'")
+ val p = Config.config.getString("akka.http.kerberos.keyTabLocation", "N/A")
+ if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.http.kerberos.keyTabLocation' is not defined in 'akka.conf'")
p
}
lazy val kerberosDebug = {
- val p = Config.config.getString("akka.rest.kerberos.kerberosDebug", "N/A")
- if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.kerberos.kerberosDebug' is not defined in 'akka.conf'")
+ val p = Config.config.getString("akka.http.kerberos.kerberosDebug", "N/A")
+ if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.http.kerberos.kerberosDebug' is not defined in 'akka.conf'")
p
}
/**
* is not used by this authenticator, so accept an empty value
*/
- lazy val realm = Config.config.getString("akka.rest.kerberos.realm", "")
+ lazy val realm = Config.config.getString("akka.http.kerberos.realm", "")
/**
* verify the kerberos token from a client with the server
diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala
index a4eb481215..d077215a7f 100644
--- a/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala
+++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala
@@ -34,10 +34,10 @@ private[akka] object CassandraStorageBackend extends CommonStorageBackend {
val REF_KEY = "item".getBytes("UTF-8")
val EMPTY_BYTE_ARRAY = new Array[Byte](0)
- val CASSANDRA_SERVER_HOSTNAME = config.getString("akka.storage.cassandra.hostname", "127.0.0.1")
- val CASSANDRA_SERVER_PORT = config.getInt("akka.storage.cassandra.port", 9160)
+ val CASSANDRA_SERVER_HOSTNAME = config.getString("akka.persistence.cassandra.hostname", "127.0.0.1")
+ val CASSANDRA_SERVER_PORT = config.getInt("akka.persistence.cassandra.port", 9160)
val CONSISTENCY_LEVEL = {
- config.getString("akka.storage.cassandra.consistency-level", "QUORUM") match {
+ config.getString("akka.persistence.cassandra.consistency-level", "QUORUM") match {
case "ZERO" => ConsistencyLevel.ZERO
case "ONE" => ConsistencyLevel.ONE
case "QUORUM" => ConsistencyLevel.QUORUM
diff --git a/akka-persistence/akka-persistence-couchdb/src/main/scala/akka/CouchDBStorageBackend.scala b/akka-persistence/akka-persistence-couchdb/src/main/scala/akka/CouchDBStorageBackend.scala
index 780222535d..e86ad9bfd1 100644
--- a/akka-persistence/akka-persistence-couchdb/src/main/scala/akka/CouchDBStorageBackend.scala
+++ b/akka-persistence/akka-persistence-couchdb/src/main/scala/akka/CouchDBStorageBackend.scala
@@ -31,8 +31,8 @@ private [akka] object CouchDBStorageBackend extends
}
lazy val URL = config.
- getString("akka.storage.couchdb.url").
- getOrElse(throw new IllegalArgumentException("'akka.storage.couchdb.url' not found in config"))
+ getString("akka.persistence.couchdb.url").
+ getOrElse(throw new IllegalArgumentException("'akka.persistence.couchdb.url' not found in config"))
def drop() = {
val client = new HttpClient()
diff --git a/akka-persistence/akka-persistence-hbase/src/main/scala/akka/HbaseStorageBackend.scala b/akka-persistence/akka-persistence-hbase/src/main/scala/akka/HbaseStorageBackend.scala
index 2b33876ecf..7e3d750803 100644
--- a/akka-persistence/akka-persistence-hbase/src/main/scala/akka/HbaseStorageBackend.scala
+++ b/akka-persistence/akka-persistence-hbase/src/main/scala/akka/HbaseStorageBackend.scala
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.util.Bytes
*/
private[akka] object HbaseStorageBackend extends MapStorageBackend[Array[Byte], Array[Byte]] with VectorStorageBackend[Array[Byte]] with RefStorageBackend[Array[Byte]] with Logging {
- val HBASE_ZOOKEEPER_QUORUM = config.getString("akka.storage.hbase.zookeeper-quorum", "localhost")
+ val HBASE_ZOOKEEPER_QUORUM = config.getString("akka.persistence.hbase.zookeeper-quorum", "localhost")
val CONFIGURATION = new HBaseConfiguration
val REF_TABLE_NAME = "__REF_TABLE"
val VECTOR_TABLE_NAME = "__VECTOR_TABLE"
diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala
index 427163d634..5e949c8a28 100644
--- a/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala
+++ b/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala
@@ -48,7 +48,7 @@ class SimpleHbaseSpecTestIntegration extends Spec with BeforeAndAfterAll with Sh
import org.apache.hadoop.hbase.client.HBaseAdmin
import org.apache.hadoop.hbase.client.HTable
- val HBASE_ZOOKEEPER_QUORUM = config.getString("akka.storage.hbase.zookeeper-quorum", "0")
+ val HBASE_ZOOKEEPER_QUORUM = config.getString("akka.persistence.hbase.zookeeper-quorum", "0")
HBASE_ZOOKEEPER_QUORUM should not equal ("0")
HBASE_ZOOKEEPER_QUORUM should equal("localhost")
diff --git a/akka-persistence/akka-persistence-memcached/src/main/scala/akka/MemcachedStorageBackend.scala b/akka-persistence/akka-persistence-memcached/src/main/scala/akka/MemcachedStorageBackend.scala
index 4b36fa4b87..0859e6d88e 100644
--- a/akka-persistence/akka-persistence-memcached/src/main/scala/akka/MemcachedStorageBackend.scala
+++ b/akka-persistence/akka-persistence-memcached/src/main/scala/akka/MemcachedStorageBackend.scala
@@ -20,7 +20,7 @@ private[akka] object MemcachedStorageBackend extends CommonStorageBackend {
import KVStorageBackend._
import org.apache.commons.codec.binary.Base64
- val clientAddresses = config.getString("akka.storage.memcached.client.addresses", "localhost:11211")
+ val clientAddresses = config.getString("akka.persistence.memcached.client.addresses", "localhost:11211")
val factory = new KetamaConnectionFactory
val client = new MemcachedClient(factory, AddrUtil.getAddresses(clientAddresses))
val base64 = new Base64(76, Array.empty[Byte], true)
@@ -114,4 +114,4 @@ private[akka] object MemcachedStorageBackend extends CommonStorageBackend {
}
-}
\ No newline at end of file
+}
diff --git a/akka-persistence/akka-persistence-mongo/src/main/scala/akka/MongoStorageBackend.scala b/akka-persistence/akka-persistence-mongo/src/main/scala/akka/MongoStorageBackend.scala
index cab7237eb0..e1a2405222 100644
--- a/akka-persistence/akka-persistence-mongo/src/main/scala/akka/MongoStorageBackend.scala
+++ b/akka-persistence/akka-persistence-mongo/src/main/scala/akka/MongoStorageBackend.scala
@@ -31,9 +31,9 @@ private[akka] object MongoStorageBackend extends
val REF = "__ref"
val COLLECTION = "akka_coll"
- val HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1")
- val DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb")
- val PORT = config.getInt("akka.storage.mongodb.port", 27017)
+ val HOSTNAME = config.getString("akka.persistence.mongodb.hostname", "127.0.0.1")
+ val DBNAME = config.getString("akka.persistence.mongodb.dbname", "testdb")
+ val PORT = config.getInt("akka.persistence.mongodb.port", 27017)
val db: MongoDB = MongoConnection(HOSTNAME, PORT)(DBNAME)
val coll: MongoCollection = db(COLLECTION)
diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/akka/RedisStorageBackend.scala b/akka-persistence/akka-persistence-redis/src/main/scala/akka/RedisStorageBackend.scala
index df7a1feab4..d6a8cfb0a7 100644
--- a/akka-persistence/akka-persistence-redis/src/main/scala/akka/RedisStorageBackend.scala
+++ b/akka-persistence/akka-persistence-redis/src/main/scala/akka/RedisStorageBackend.scala
@@ -52,14 +52,14 @@ private [akka] object RedisStorageBackend extends
Logging {
// need an explicit definition in akka-conf
- val nodes = config.getList("akka.storage.redis.cluster")
+ val nodes = config.getList("akka.persistence.redis.cluster")
def connect() =
nodes match {
case Seq() =>
// no cluster defined
- val REDIS_SERVER_HOSTNAME = config.getString("akka.storage.redis.hostname", "127.0.0.1")
- val REDIS_SERVER_PORT = config.getInt("akka.storage.redis.port", 6379)
+ val REDIS_SERVER_HOSTNAME = config.getString("akka.persistence.redis.hostname", "127.0.0.1")
+ val REDIS_SERVER_PORT = config.getInt("akka.persistence.redis.port", 6379)
new RedisClient(REDIS_SERVER_HOSTNAME, REDIS_SERVER_PORT)
case s =>
diff --git a/akka-persistence/akka-persistence-riak/src/main/scala/akka/RiakStorageBackend.scala b/akka-persistence/akka-persistence-riak/src/main/scala/akka/RiakStorageBackend.scala
index 57c8776a1b..149576da05 100644
--- a/akka-persistence/akka-persistence-riak/src/main/scala/akka/RiakStorageBackend.scala
+++ b/akka-persistence/akka-persistence-riak/src/main/scala/akka/RiakStorageBackend.scala
@@ -18,12 +18,12 @@ import com.trifork.riak.{RequestMeta, RiakObject, RiakClient}
private[akka] object RiakStorageBackend extends CommonStorageBackend {
- val refBucket = config.getString("akka.storage.riak.bucket.ref", "Refs")
- val mapBucket = config.getString("akka.storage.riak.bucket.map", "Maps")
- val vectorBucket = config.getString("akka.storage.riak.bucket.vector", "Vectors")
- val queueBucket = config.getString("akka.storage.riak.bucket.queue", "Queues")
- val clientHost = config.getString("akka.storage.riak.client.host", "localhost")
- val clientPort = config.getInt("akka.storage.riak.client.port", 8087)
+ val refBucket = config.getString("akka.persistence.riak.bucket.ref", "Refs")
+ val mapBucket = config.getString("akka.persistence.riak.bucket.map", "Maps")
+ val vectorBucket = config.getString("akka.persistence.riak.bucket.vector", "Vectors")
+ val queueBucket = config.getString("akka.persistence.riak.bucket.queue", "Queues")
+ val clientHost = config.getString("akka.persistence.riak.client.host", "localhost")
+ val clientPort = config.getInt("akka.persistence.riak.client.port", 8087)
val riakClient: RiakClient = new RiakClient(clientHost, clientPort);
import CommonStorageBackendAccess._
diff --git a/akka-persistence/akka-persistence-simpledb/src/main/scala/akka/SimpledbStorageBackend.scala b/akka-persistence/akka-persistence-simpledb/src/main/scala/akka/SimpledbStorageBackend.scala
index f0441a60ef..dba6579e41 100644
--- a/akka-persistence/akka-persistence-simpledb/src/main/scala/akka/SimpledbStorageBackend.scala
+++ b/akka-persistence/akka-persistence-simpledb/src/main/scala/akka/SimpledbStorageBackend.scala
@@ -28,42 +28,42 @@ private[akka] object SimpledbStorageBackend extends CommonStorageBackend {
val ownerAtt = "owner"
val base64 = new Base64(1024, seperatorBytes, true)
val base64key = new Base64(1024, Array.empty[Byte], true)
- val id = config.getString("akka.storage.simpledb.account.id").getOrElse{
+ val id = config.getString("akka.persistence.simpledb.account.id").getOrElse{
val e = new IllegalStateException("You must provide an AWS id")
log.error(e, "You Must Provide an AWS id to use the SimpledbStorageBackend")
throw e
}
- val secretKey = config.getString("akka.storage.simpledb.account.secretKey").getOrElse{
+ val secretKey = config.getString("akka.persistence.simpledb.account.secretKey").getOrElse{
val e = new IllegalStateException("You must provide an AWS secretKey")
log.error(e, "You Must Provide an AWS secretKey to use the SimpledbStorageBackend")
throw e
}
- val refDomain = config.getString("akka.storage.simpledb.domain.ref", "ref")
- val mapDomain = config.getString("akka.storage.simpledb.domain.map", "map")
- val queueDomain = config.getString("akka.storage.simpledb.domain.queue", "queue")
- val vectorDomain = config.getString("akka.storage.simpledb.domain.vector", "vector")
+ val refDomain = config.getString("akka.persistence.simpledb.domain.ref", "ref")
+ val mapDomain = config.getString("akka.persistence.simpledb.domain.map", "map")
+ val queueDomain = config.getString("akka.persistence.simpledb.domain.queue", "queue")
+ val vectorDomain = config.getString("akka.persistence.simpledb.domain.vector", "vector")
val credentials = new BasicAWSCredentials(id, secretKey);
val clientConfig = new ClientConfiguration()
- for (i <- config.getInt("akka.storage.simpledb.client.timeout")) {
+ for (i <- config.getInt("akka.persistence.simpledb.client.timeout")) {
clientConfig.setConnectionTimeout(i)
}
- for (i <- config.getInt("akka.storage.simpledb.client.maxconnections")) {
+ for (i <- config.getInt("akka.persistence.simpledb.client.maxconnections")) {
clientConfig.setMaxConnections(i)
}
- clientConfig.setMaxErrorRetry(config.getInt("akka.storage.simpledb.client.maxretries", 10))
+ clientConfig.setMaxErrorRetry(config.getInt("akka.persistence.simpledb.client.maxretries", 10))
- for (s <- config.getString("akka.storage.simpledb.client.protocol")) {
+ for (s <- config.getString("akka.persistence.simpledb.client.protocol")) {
clientConfig.setProtocol(Protocol.valueOf(s))
}
- for (i <- config.getInt("akka.storage.simpledb.client.sockettimeout")) {
+ for (i <- config.getInt("akka.persistence.simpledb.client.sockettimeout")) {
clientConfig.setSocketTimeout(i)
}
- for {s <- config.getInt("akka.storage.simpledb.client.sendbuffer")
- r <- config.getInt("akka.storage.simpledb.client.receivebuffer")} {
+ for {s <- config.getInt("akka.persistence.simpledb.client.sendbuffer")
+ r <- config.getInt("akka.persistence.simpledb.client.receivebuffer")} {
clientConfig.setSocketBufferSizeHints(s, r)
}
- for (s <- config.getString("akka.storage.simpledb.client.useragent")) {
+ for (s <- config.getString("akka.persistence.simpledb.client.useragent")) {
clientConfig.setUserAgent(s)
}
@@ -292,4 +292,4 @@ private[akka] object SimpledbStorageBackend extends CommonStorageBackend {
}
-}
\ No newline at end of file
+}
diff --git a/akka-persistence/akka-persistence-voldemort/src/main/scala/akka/VoldemortStorageBackend.scala b/akka-persistence/akka-persistence-voldemort/src/main/scala/akka/VoldemortStorageBackend.scala
index 520b2030fe..8f2779eb06 100644
--- a/akka-persistence/akka-persistence-voldemort/src/main/scala/akka/VoldemortStorageBackend.scala
+++ b/akka-persistence/akka-persistence-voldemort/src/main/scala/akka/VoldemortStorageBackend.scala
@@ -28,14 +28,14 @@ private[akka] object VoldemortStorageBackend extends CommonStorageBackend {
import VoldemortAccess._
val bootstrapUrlsProp = "bootstrap_urls"
- val clientConfig = config.getConfigMap("akka.storage.voldemort.client") match {
+ val clientConfig = config.getConfigMap("akka.persistence.voldemort.client") match {
case Some(configMap) => getClientConfig(configMap.asMap)
case None => getClientConfig(new HashMap[String, String] + (bootstrapUrlsProp -> "tcp://localhost:6666"))
}
- val refStore = config.getString("akka.storage.voldemort.store.ref", "Refs")
- val mapStore = config.getString("akka.storage.voldemort.store.map", "Maps")
- val vectorStore = config.getString("akka.storage.voldemort.store.vector", "Vectors")
- val queueStore = config.getString("akka.storage.voldemort.store.queue", "Queues")
+ val refStore = config.getString("akka.persistence.voldemort.store.ref", "Refs")
+ val mapStore = config.getString("akka.persistence.voldemort.store.map", "Maps")
+ val vectorStore = config.getString("akka.persistence.voldemort.store.vector", "Vectors")
+ val queueStore = config.getString("akka.persistence.voldemort.store.queue", "Queues")
var storeClientFactory: StoreClientFactory = null
diff --git a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala
index 506d95905b..d67c09cd93 100644
--- a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala
+++ b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala
@@ -22,11 +22,11 @@ trait BootableRemoteActorService extends Bootable with Logging {
else RemoteNode.start
}
}, "Akka Remote Service")
-
+
def startRemoteService = remoteServerThread.start
abstract override def onLoad = {
- if (config.getBool("akka.remote.server.service", true)) {
+ if (RemoteServer.isRemotingEnabled) {
log.info("Initializing Remote Actors Service...")
startRemoteService
log.info("Remote Actors Service initialized")
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteClient.scala b/akka-remote/src/main/scala/akka/remote/RemoteClient.scala
index e80c7936c2..e501411f6f 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteClient.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteClient.scala
@@ -417,7 +417,7 @@ class RemoteClientHandler(
if (result.isInstanceOf[RemoteMessageProtocol]) {
val reply = result.asInstanceOf[RemoteMessageProtocol]
val replyUuid = uuidFrom(reply.getUuid.getHigh, reply.getUuid.getLow)
- log.debug("Remote client received RemoteMessageProtocol[\n%s]", reply.toString)
+ log.debug("Remote client received RemoteMessageProtocol[\n%s]".format(reply.toString))
val future = futures.get(replyUuid).asInstanceOf[CompletableFuture[Any]]
if (reply.hasMessage) {
if (future eq null) throw new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist")
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteServer.scala b/akka-remote/src/main/scala/akka/remote/RemoteServer.scala
index 1d79978559..a7c23a8d06 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteServer.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteServer.scala
@@ -66,12 +66,14 @@ object RemoteNode extends RemoteServer
* @author Jonas Bonér
*/
object RemoteServer {
+ val isRemotingEnabled = config.getList("akka.enabled-modules").exists(_ == "remote")
+
val UUID_PREFIX = "uuid:"
val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.server.message-frame-size", 1048576)
val SECURE_COOKIE = config.getString("akka.remote.secure-cookie")
val REQUIRE_COOKIE = {
val requireCookie = config.getBool("akka.remote.server.require-cookie", true)
- if (requireCookie && RemoteServer.SECURE_COOKIE.isEmpty) throw new ConfigurationException(
+ if (isRemotingEnabled && requireCookie && RemoteServer.SECURE_COOKIE.isEmpty) throw new ConfigurationException(
"Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.")
requireCookie
}
diff --git a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala
index f713f6035d..7026d1cfc0 100644
--- a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala
+++ b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala
@@ -7,7 +7,7 @@ import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import akka.serialization.Serializer.ScalaJSON
-
+/*
object Protocols {
import sjson.json.DefaultProtocol._
case class Shop(store: String, item: String, price: Int)
@@ -50,3 +50,4 @@ class ScalaJSONSerializerSpec extends
}
}
}
+*/
diff --git a/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala b/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala
index 1a462d1700..bee8dd5bd9 100644
--- a/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala
+++ b/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala
@@ -39,7 +39,7 @@ class Boot {
}
/*
- * In akka.conf you can set the FQN of any AuthenticationActor of your wish, under the property name: akka.rest.authenticator
+ * In akka.conf you can set the FQN of any AuthenticationActor of your wish, under the property name: akka.http.authenticator
*/
class DigestAuthenticationService extends DigestAuthenticationActor {
//If you want to have a distributed nonce-map, you can use something like below,
diff --git a/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml b/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml
index e9d5bbb4db..d532f0e3f2 100644
--- a/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml
+++ b/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml
@@ -8,7 +8,7 @@
AkkaServlet
- akka.rest.AkkaServlet
+ akka.http.AkkaServlet
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala
new file mode 100644
index 0000000000..31c17a725d
--- /dev/null
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala
@@ -0,0 +1,26 @@
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+object TypedActorRegistrySpec {
+ trait My
+ class MyImpl extends TypedActor with My
+}
+
+class TypedActorRegistrySpec extends JUnitSuite {
+ import TypedActorRegistrySpec._
+
+ @Test def shouldGetTypedActorByClassFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val my = TypedActor.newInstance[My](classOf[My], classOf[MyImpl], 3000)
+
+ val actors = ActorRegistry.typedActorsFor(classOf[My])
+ assert(actors.length === 1)
+
+ val option = ActorRegistry.typedActorFor[My]
+ assert(option != null)
+ assert(option.isDefined)
+ ActorRegistry.shutdownAll
+ }
+}
diff --git a/config/akka-reference.conf b/config/akka-reference.conf
index dd28994ed2..42a4f1f06b 100644
--- a/config/akka-reference.conf
+++ b/config/akka-reference.conf
@@ -8,7 +8,9 @@
akka {
version = "1.0-SNAPSHOT" # Akka version, checked against the runtime version of Akka.
- time-unit = "seconds" # Default timeout time unit for all timeout properties throughout the config
+ enabled-modules = [] # Comma separated list of the enabled modules. Options: ["remote", "camel", "http"]
+
+ time-unit = "seconds" # Time unit for all timeout properties throughout the config
# These boot classes are loaded (and created) automatically when the Akka Microkernel boots up
# Can be used to bootstrap your application(s)
@@ -84,8 +86,7 @@ akka {
timeout = 60
}
- rest {
- service = on
+ http {
hostname = "localhost"
port = 9998
#cometSupport = "org.atmosphere.container.Jetty7CometSupport" # Disregard autodetection, for valid values: http://doc.akkasource.org/comet
@@ -123,30 +124,13 @@ akka {
compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
- ssl {
- service = off # NOTE: This feature is not deemed production ready and is not possible to turn on yet
-
- # You can either use java command-line options or use the settings below
-
- #key-store-type = "pkcs12" # Same as -Djavax.net.ssl.keyStoreType=pkcs12
- #key-store = "yourcertificate.p12" # Same as -Djavax.net.ssl.keyStore=yourcertificate.p12
- #key-store-pass = "$PASS" # Same as -Djavax.net.ssl.keyStorePassword=$PASS
-
- #trust-store-type = "jks" # Same as -Djavax.net.ssl.trustStoreType=jks
- #trust-store = "your.keystore" # Same as -Djavax.net.ssl.trustStore=your.keystore
- #trust-store-pass = "$PASS" # Same as -Djavax.net.ssl.trustStorePassword=$PASS
-
- debug = off # This can be useful for debugging. If on, very verbose debug, same as -Djavax.net.debug=ssl
- }
-
server {
- service = on
- hostname = "localhost" # The hostname or IP that clients should connect to
- port = 2552 # The port clients should connect to
- message-frame-size = 1048576
+ hostname = "localhost" # The hostname or IP that clients should connect to
+ port = 2552 # The port clients should connect to. Default is 2552 (AKKA)
+ message-frame-size = 1048576 # Increase this if you want to be able to send messages with large payloads
connection-timeout = 1
- require-cookie = on
- untrusted-mode = off
+ require-cookie = on # Should the remote server require that it peers share the same secure-cookie (defined in the 'remote' section)?
+ untrusted-mode = off # Enable untrusted mode for full security of server managed actors, allows untrusted clients to connect.
}
client {
@@ -155,15 +139,9 @@ akka {
message-frame-size = 1048576
reconnection-time-window = 600 # Maximum time window that a client should try to reconnect for
}
-
- cluster {
- service = on
- name = "default" # The name of the cluster
- serializer = "akka.serialization.Serializer$Java$" # FQN of the serializer class
- }
}
- storage {
+ persistence {
cassandra {
hostname = "127.0.0.1" # IP address or hostname of one of the Cassandra cluster's seeds
port = 9160 # Port to Cassandra
@@ -210,24 +188,25 @@ akka {
queue = "Queues"
}
- client{
+ client {
host = "localhost"
port = 8087 #Default Riak Protobuf port
}
}
memcached {
- client{
- addresses = "localhost:11211" #Formatted according to spymemcached "localhost:11211 otherhost:11211" etc..
+ client {
+ addresses = "localhost:11211" #Formatted according to spymemcached "localhost:11211 otherhost:11211" etc..
}
}
simpledb {
- account{
+ account {
id = "YOU NEED TO PROVIDE AN AWS ID"
secretKey = "YOU NEED TO PROVIDE AN AWS SECRETKEY"
}
- client{
+
+ client {
#Defaults to default AWS ClientConfiguration
timeout =50000
#maxconnections =
@@ -238,7 +217,8 @@ akka {
#receivebuffer = 0
#useragent = "AWS Java SDK-1.0.14"
}
- domain{
+
+ domain {
ref = "ref"
map = "map"
vector = "vector"
@@ -246,8 +226,4 @@ akka {
}
}
}
-
- camel {
- service = on
- }
}
diff --git a/project/build.properties b/project/build.properties
index 4b70e4a190..b09217e854 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -1,7 +1,7 @@
project.organization=se.scalablesolutions.akka
project.name=akka
project.version=1.0-SNAPSHOT
-scala.version=2.8.0
+scala.version=2.8.1
sbt.version=0.7.4
def.scala.version=2.7.7
-build.scala.versions=2.8.0
+build.scala.versions=2.8.1
diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index e24843577b..94225f90fb 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -21,7 +21,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
"-Xmigration",
"-Xcheckinit",
"-Xstrict-warnings",
- // "-optimise", //Uncomment this for release compile
+ // "-optimise", //Uncomment this for release compile
"-Xwarninit",
"-encoding", "utf8")
.map(CompileOption(_))
@@ -262,7 +262,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val junit = "junit" % "junit" % "4.5" % "test" //Common Public License 1.0
lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" //MIT
lazy val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test" //ApacheV2
- lazy val specs = "org.scala-tools.testing" %% "specs" % "1.6.5" % "test" //MIT
+ lazy val specs = "org.scala-tools.testing" %% "specs" % "1.6.6" % "test" //MIT
//HBase testing
lazy val hadoop_test = "org.apache.hadoop" % "hadoop-test" % "0.20.2" % "test" //ApacheV2
@@ -323,7 +323,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
.filter(_.getName.endsWith(".jar"))
.filter(!_.getName.contains("servlet_2.4"))
.filter(!_.getName.contains("scala-library"))
- .map("lib_managed/scala_%s/compile/".format(buildScalaVersion) + _.getName)
+ .map("lib_managed/compile/" + _.getName)
.mkString(" ") +
" config/" +
" scala-library.jar" +
@@ -535,6 +535,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// -------------------------------------------------------------------------------------------------------------------
class AkkaPersistenceParentProject(info: ProjectInfo) extends ParentProject(info) {
+ override def disableCrossPaths = true
+
lazy val akka_persistence_common = project("akka-persistence-common", "akka-persistence-common",
new AkkaPersistenceCommonProject(_), akka_remote, akka_stm)
lazy val akka_persistence_redis = project("akka-persistence-redis", "akka-persistence-redis",
@@ -742,6 +744,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// -------------------------------------------------------------------------------------------------------------------
class AkkaOSGiParentProject(info: ProjectInfo) extends ParentProject(info) {
+ override def disableCrossPaths = true
+
lazy val akka_osgi_dependencies_bundle = project("akka-osgi-dependencies-bundle", "akka-osgi-dependencies-bundle",
new AkkaOSGiDependenciesBundleProject(_), akka_kernel, akka_jta) // akka_kernel does not depend on akka_jta (why?) therefore we list akka_jta here
lazy val akka_osgi_assembly = project("akka-osgi-assembly", "akka-osgi-assembly",
@@ -778,6 +782,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
}
class AkkaOSGiAssemblyProject(info: ProjectInfo) extends DefaultProject(info) {
+ override def disableCrossPaths = true
// Scala bundle
val scala_bundle = "com.weiglewilczek.scala-lang-osgi" % "scala-library" % buildScalaVersion % "compile" intransitive
@@ -838,6 +843,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// -------------------------------------------------------------------------------------------------------------------
class AkkaSampleAntsProject(info: ProjectInfo) extends DefaultSpdeProject(info) {
+ override def disableCrossPaths = true
override def spdeSourcePath = mainSourcePath / "spde"
}
@@ -886,6 +892,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
}
class AkkaSamplesParentProject(info: ProjectInfo) extends ParentProject(info) {
+ override def disableCrossPaths = true
+
lazy val akka_sample_ants = project("akka-sample-ants", "akka-sample-ants",
new AkkaSampleAntsProject(_), akka_stm)
lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat",
diff --git a/project/plugins/Plugins.scala b/project/plugins/Plugins.scala
index 4f4ec09479..0a2e64a2a8 100644
--- a/project/plugins/Plugins.scala
+++ b/project/plugins/Plugins.scala
@@ -25,5 +25,5 @@ class Plugins(info: ProjectInfo) extends PluginDefinition(info) {
// Dependencies
// -------------------------------------------------------------------------------------------------------------------
lazy val bnd4sbt = "com.weiglewilczek.bnd4sbt" % "bnd4sbt" % "1.0.0.RC4"
- lazy val spdeSbt = "us.technically.spde" % "spde-sbt-plugin" % "0.4.1"
+ lazy val spdeSbt = "us.technically.spde" % "spde-sbt-plugin" % "0.4.2"
}