diff --git a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java
index 7f552c46fc..eb972def86 100644
--- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java
+++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java
@@ -20,7 +20,7 @@ import scala.Right;
public class JavaFutureTests {
private final ActorSystem app = ActorSystem.create();
- private final Timeout t = app.AkkaConfig().ActorTimeout();
+ private final Timeout t = app.settings().ActorTimeout();
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);
@Test public void mustBeAbleToMapAFuture() {
diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala
index 5d8333a6cd..0057cdda60 100644
--- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala
+++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala
@@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
}
}, timeout = t))
- val defaultTimeout = system.AkkaConfig.ActorTimeout.duration
- val testTimeout = if (system.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
+ val defaultTimeout = system.settings.ActorTimeout.duration
+ val testTimeout = if (system.settings.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
"An Actor-based Future" must {
diff --git a/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala
index 09afd0b527..a96fc68435 100644
--- a/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala
+++ b/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala
@@ -9,7 +9,7 @@ class ClusterSpec extends AkkaSpec {
"be able to parse 'akka.actor.cluster._' config elements" in {
// TODO: make it use its own special config?
- val config = system.AkkaConfig.config
+ val config = system.settings.config
import config._
//akka.cluster
diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala
index a1b991add1..aa6e4dc7c9 100644
--- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala
+++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala
@@ -421,7 +421,7 @@ class DispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
- new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.AkkaConfig.DispatcherThroughput,
+ new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.settings.DispatcherThroughput,
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala
index ffdc34a903..d33fb9730f 100644
--- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala
+++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala
@@ -13,7 +13,7 @@ class ConfigSpec extends AkkaSpec(ActorSystem("ConfigSpec", Configuration.fromFi
"The default configuration file (i.e. akka-reference.conf)" must {
"contain all configuration properties for akka-actor that are used in code with their correct defaults" in {
- val config = system.AkkaConfig.config
+ val config = system.settings.config
import config._
getList("akka.boot") must equal(Nil)
diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala
index 7ed96a6dc8..d73009377a 100644
--- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala
+++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala
@@ -18,7 +18,7 @@ class PriorityDispatcherSpec extends AkkaSpec {
testOrdering(BoundedPriorityMailbox(PriorityGenerator({
case i: Int ⇒ i //Reverse order
case 'Result ⇒ Int.MaxValue
- }: Any ⇒ Int), 1000, system.AkkaConfig.MailboxPushTimeout))
+ }: Any ⇒ Int), 1000, system.settings.MailboxPushTimeout))
}
}
diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala
index 05500932dd..f52e2d8d60 100644
--- a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala
+++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala
@@ -221,11 +221,11 @@ class Report(
sb.append("Args:\n ").append(args)
sb.append("\n")
- sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
+ sb.append("Akka version: ").append(app.settings.ConfigVersion)
sb.append("\n")
sb.append("Akka config:")
- for (key ← app.AkkaConfig.config.keys) {
- sb.append("\n ").append(key).append("=").append(app.AkkaConfig.config(key))
+ for (key ← app.settings.config.keys) {
+ sb.append("\n ").append(key).append("=").append(app.settings.config(key))
}
sb.toString
diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala
index b9aba3ac1a..28334980b4 100644
--- a/akka-actor/src/main/scala/akka/actor/Actor.scala
+++ b/akka-actor/src/main/scala/akka/actor/Actor.scala
@@ -150,7 +150,7 @@ object Timeout {
implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
- implicit def defaultTimeout(implicit app: ActorSystem) = app.AkkaConfig.ActorTimeout
+ implicit def defaultTimeout(implicit app: ActorSystem) = app.settings.ActorTimeout
}
trait ActorLogging { this: Actor ⇒
@@ -234,7 +234,7 @@ trait Actor {
/**
* The default timeout, based on the config setting 'akka.actor.timeout'
*/
- implicit def defaultTimeout = system.AkkaConfig.ActorTimeout
+ implicit def defaultTimeout = system.settings.ActorTimeout
/**
* Wrap a Receive partial function in a logging enclosure, which sends a
@@ -250,7 +250,7 @@ trait Actor {
* This method does NOT modify the given Receive unless
* akka.actor.debug.receive is set within akka.conf.
*/
- def loggable(self: AnyRef)(r: Receive): Receive = if (system.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
+ def loggable(self: AnyRef)(r: Receive): Receive = if (system.settings.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
/**
* Some[ActorRef] representation of the 'self' ActorRef reference.
diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala
index fd7110a8de..052cf47cb0 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala
@@ -174,7 +174,7 @@ private[akka] class ActorCell(
actor = created
created.preStart()
checkReceiveTimeout
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")"))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")"))
} catch {
case e ⇒
try {
@@ -188,7 +188,7 @@ private[akka] class ActorCell(
def recreate(cause: Throwable): Unit = try {
val failedActor = actor
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting"))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting"))
val freshActor = newActor()
if (failedActor ne null) {
val c = currentMessage //One read only plz
@@ -202,7 +202,7 @@ private[akka] class ActorCell(
}
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
freshActor.postRestart(cause)
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted"))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted"))
dispatcher.resume(this) //FIXME should this be moved down?
@@ -228,7 +228,7 @@ private[akka] class ActorCell(
val c = children
if (c.isEmpty) doTerminate()
else {
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping"))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping"))
for (child ← c) child.stop()
stopping = true
}
@@ -239,7 +239,7 @@ private[akka] class ActorCell(
if (!stats.contains(child)) {
childrenRefs = childrenRefs.updated(child.name, child)
childrenStats = childrenStats.updated(child, ChildRestartStats())
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child))
} else app.eventStream.publish(Warning(self, "Already supervising " + child))
}
@@ -255,10 +255,10 @@ private[akka] class ActorCell(
case Recreate(cause) ⇒ recreate(cause)
case Link(subject) ⇒
app.deathWatch.subscribe(self, subject)
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject))
case Unlink(subject) ⇒
app.deathWatch.unsubscribe(self, subject)
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject))
case Suspend() ⇒ suspend()
case Resume() ⇒ resume()
case Terminate() ⇒ terminate()
@@ -332,7 +332,7 @@ private[akka] class ActorCell(
}
def autoReceiveMessage(msg: Envelope) {
- if (app.AkkaConfig.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg))
+ if (app.settings.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg))
if (stopping) msg.message match {
case ChildTerminated ⇒ handleChildTerminated(sender)
@@ -359,7 +359,7 @@ private[akka] class ActorCell(
try {
parent.tell(ChildTerminated, self)
app.deathWatch.publish(Terminated(self))
- if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped"))
+ if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped"))
} finally {
currentMessage = null
clearActorContext()
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala
index 09aef9b570..5b45421afe 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala
@@ -34,7 +34,7 @@ trait ActorRefProvider {
def nodename: String
- def AkkaConfig: ActorSystem.AkkaConfig
+ def settings: ActorSystem.Settings
def init(app: ActorSystemImpl)
@@ -126,7 +126,7 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
* Local ActorRef provider.
*/
class LocalActorRefProvider(
- val AkkaConfig: ActorSystem.AkkaConfig,
+ val settings: ActorSystem.Settings,
val rootPath: ActorPath,
val eventStream: EventStream,
val dispatcher: MessageDispatcher,
@@ -139,7 +139,7 @@ class LocalActorRefProvider(
case value ⇒ value
}
- private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename)
+ private[akka] val deployer: Deployer = new Deployer(settings, eventStream, nodename)
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher)
@@ -290,7 +290,7 @@ class LocalActorRefProvider(
case RouterType.Random ⇒ () ⇒ new RandomRouter
case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter
case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()(
- if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, AkkaConfig.ActorTimeout)
+ if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, settings.ActorTimeout)
case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet")
@@ -350,7 +350,7 @@ class LocalActorRefProvider(
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = {
import akka.dispatch.{ Future, Promise, DefaultPromise }
- (if (within == null) AkkaConfig.ActorTimeout else within) match {
+ (if (within == null) settings.ActorTimeout else within) match {
case t if t.duration.length <= 0 ⇒ new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout
case t ⇒
val a = new AskActorRef(tempPath, this, deathWatch, t, dispatcher) { def whenDone() = actors.remove(this) }
diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala
index 70664c9f46..3ad7ddf5f5 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala
@@ -74,7 +74,7 @@ object ActorSystem {
case object Stopped extends ExitStatus
case class Failed(cause: Throwable) extends ExitStatus
- class AkkaConfig(val config: Configuration) {
+ class Settings(val config: Configuration) {
import config._
val ConfigVersion = getString("akka.version", Version)
@@ -133,7 +133,7 @@ abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
import ActorSystem._
def name: String
- def AkkaConfig: AkkaConfig
+ def settings: Settings
def nodename: String
/**
@@ -169,25 +169,25 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst
import ActorSystem._
- val AkkaConfig = new AkkaConfig(config)
+ val settings = new Settings(config)
protected def app = this
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true)
- import AkkaConfig._
+ import settings._
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
case null | "" ⇒ InetAddress.getLocalHost.getHostAddress
case value ⇒ value
}, System.getProperty("akka.remote.port") match {
- case null | "" ⇒ AkkaConfig.RemoteServerPort
+ case null | "" ⇒ settings.RemoteServerPort
case value ⇒ value.toInt
})
// this provides basic logging (to stdout) until .start() is called below
val eventStream = new EventStream(DebugEventStream)
- eventStream.startStdoutLogger(AkkaConfig)
+ eventStream.startStdoutLogger(settings)
val log = new BusLogging(eventStream, this) // “this” used only for .getClass in tagging messages
/**
@@ -211,7 +211,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst
val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
// TODO correctly pull its config from the config
- val dispatcherFactory = new Dispatchers(AkkaConfig, eventStream, deadLetterMailbox, scheduler)
+ val dispatcherFactory = new Dispatchers(settings, eventStream, deadLetterMailbox, scheduler)
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
deadLetters.init(dispatcher)
@@ -222,7 +222,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst
case Right(b) ⇒ b
}
val arguments = List(
- classOf[AkkaConfig] -> AkkaConfig,
+ classOf[Settings] -> settings,
classOf[ActorPath] -> rootPath,
classOf[EventStream] -> eventStream,
classOf[MessageDispatcher] -> dispatcher,
@@ -256,7 +256,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst
def start(): this.type = {
_serialization = new Serialization(this)
- _typedActor = new TypedActor(AkkaConfig, _serialization)
+ _typedActor = new TypedActor(settings, _serialization)
provider.init(this)
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
eventStream.start(this)
diff --git a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
index 0876671d6e..3dd15e02e5 100644
--- a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
+++ b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
@@ -16,12 +16,12 @@ trait BootableActorLoaderService extends Bootable {
def app: ActorSystem
- val BOOT_CLASSES = app.AkkaConfig.BootClasses
+ val BOOT_CLASSES = app.settings.BootClasses
lazy val applicationLoader = createApplicationClassLoader()
protected def createApplicationClassLoader(): Option[ClassLoader] = Some({
- if (app.AkkaConfig.Home.isDefined) {
- val DEPLOY = app.AkkaConfig.Home.get + "/deploy"
+ if (app.settings.Home.isDefined) {
+ val DEPLOY = app.settings.Home.get + "/deploy"
val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) {
System.exit(-1)
diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala
index 68b2d98d61..4b5d64bde6 100644
--- a/akka-actor/src/main/scala/akka/actor/Deployer.scala
+++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala
@@ -33,7 +33,7 @@ trait ActorDeployer {
*
* @author Jonas Bonér
*/
-class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStream, val nodename: String) extends ActorDeployer {
+class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream, val nodename: String) extends ActorDeployer {
val deploymentConfig = new DeploymentConfig(nodename)
val log = Logging(eventStream, this)
@@ -85,7 +85,7 @@ class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStr
private[akka] def pathsInConfig: List[String] = {
val deploymentPath = "akka.actor.deployment"
- AkkaConfig.config.getSection(deploymentPath) match {
+ settings.config.getSection(deploymentPath) match {
case None ⇒ Nil
case Some(pathConfig) ⇒
pathConfig.map.keySet
@@ -97,7 +97,7 @@ class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStr
/**
* Lookup deployment in 'akka.conf' configuration file.
*/
- private[akka] def lookupInConfig(path: String, configuration: Configuration = AkkaConfig.config): Option[Deploy] = {
+ private[akka] def lookupInConfig(path: String, configuration: Configuration = settings.config): Option[Deploy] = {
import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor }
// --------------------------------
diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala
index 79ca427be4..32c34a801f 100644
--- a/akka-actor/src/main/scala/akka/actor/FSM.scala
+++ b/akka-actor/src/main/scala/akka/actor/FSM.scala
@@ -565,7 +565,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
def logDepth: Int = 0
- private val debugEvent = system.AkkaConfig.FsmDebugEvent
+ private val debugEvent = system.settings.FsmDebugEvent
private val events = new Array[Event](logDepth)
private val states = new Array[AnyRef](logDepth)
diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala
index ad4b074cef..51bf2e4639 100644
--- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala
+++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala
@@ -116,7 +116,7 @@ object TypedActor {
/**
* Returns the default timeout (for a TypedActor) when inside a method call in a TypedActor.
*/
- implicit def timeout = app.AkkaConfig.ActorTimeout
+ implicit def timeout = app.settings.ActorTimeout
}
trait TypedActorFactory { this: ActorRefFactory ⇒
@@ -264,7 +264,7 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
*
* TypedActors needs, just like Actors, to be Stopped when they are no longer needed, use TypedActor.stop(proxy)
*/
-class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization) {
+class TypedActor(val settings: ActorSystem.Settings, var ser: Serialization) {
import TypedActor.MethodCall
/**
@@ -313,7 +313,7 @@ class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization)
//Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
val actorVar = new AtomVar[ActorRef](null)
val timeout = props.timeout match {
- case Props.`defaultTimeout` ⇒ AkkaConfig.ActorTimeout
+ case Props.`defaultTimeout` ⇒ settings.ActorTimeout
case x ⇒ x
}
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T]
diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala
index 37c75716d5..db9f3dc183 100644
--- a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala
+++ b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala
@@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) {
*/
object NodeAddress {
def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName)
- def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename)
+ def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.settings.ClusterName, app.nodename)
def unapply(other: Any) = other match {
case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName))
diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala
index 4df4eeac39..04fd0f94f2 100644
--- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala
@@ -14,7 +14,7 @@ import akka.actor._
import akka.actor.ActorSystem
import scala.annotation.tailrec
import akka.event.EventStream
-import akka.actor.ActorSystem.AkkaConfig
+import akka.actor.ActorSystem.Settings
/**
* @author Jonas Bonér
@@ -330,19 +330,19 @@ abstract class MessageDispatcher(
/**
* Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig
*/
-abstract class MessageDispatcherConfigurator(val AkkaConfig: AkkaConfig, val eventStream: EventStream) {
+abstract class MessageDispatcherConfigurator(val settings: Settings, val eventStream: EventStream) {
/**
* Returns an instance of MessageDispatcher given a Configuration
*/
def configure(config: Configuration): MessageDispatcher
def mailboxType(config: Configuration): MailboxType = {
- val capacity = config.getInt("mailbox-capacity", AkkaConfig.MailboxCapacity)
+ val capacity = config.getInt("mailbox-capacity", settings.MailboxCapacity)
if (capacity < 1) UnboundedMailbox()
else {
val duration = Duration(
- config.getInt("mailbox-push-timeout-time", AkkaConfig.MailboxPushTimeout.toMillis.toInt),
- AkkaConfig.DefaultTimeUnit)
+ config.getInt("mailbox-push-timeout-time", settings.MailboxPushTimeout.toMillis.toInt),
+ settings.DefaultTimeUnit)
BoundedMailbox(capacity, duration)
}
}
@@ -352,7 +352,7 @@ abstract class MessageDispatcherConfigurator(val AkkaConfig: AkkaConfig, val eve
//Apply the following options to the config if they are present in the config
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(eventStream)).configure(
- conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, AkkaConfig.DefaultTimeUnit))),
+ conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, settings.DefaultTimeUnit))),
conf_?(config getDouble "core-pool-size-factor")(factor ⇒ _.setCorePoolSizeFromFactor(factor)),
conf_?(config getDouble "max-pool-size-factor")(factor ⇒ _.setMaxPoolSizeFromFactor(factor)),
conf_?(config getInt "executor-bounds")(bounds ⇒ _.setExecutorBounds(bounds)),
diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
index 654c3b338e..c94ab9f02c 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
@@ -12,7 +12,7 @@ import java.util.concurrent.TimeUnit
import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
-import akka.actor.ActorSystem.AkkaConfig
+import akka.actor.ActorSystem.Settings
/**
* Scala API. Dispatcher factory.
@@ -47,19 +47,19 @@ import akka.actor.ActorSystem.AkkaConfig
* @author Jonas Bonér
*/
class Dispatchers(
- val AkkaConfig: ActorSystem.AkkaConfig,
+ val settings: ActorSystem.Settings,
val eventStream: EventStream,
val deadLetterMailbox: Mailbox,
val scheduler: Scheduler) {
- val ThroughputDeadlineTimeMillis = AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
+ val ThroughputDeadlineTimeMillis = settings.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType =
- if (AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
- else BoundedMailbox(AkkaConfig.MailboxCapacity, AkkaConfig.MailboxPushTimeout)
- val DispatcherShutdownMillis = AkkaConfig.DispatcherDefaultShutdown.toMillis
+ if (settings.MailboxCapacity < 1) UnboundedMailbox()
+ else BoundedMailbox(settings.MailboxCapacity, settings.MailboxPushTimeout)
+ val DispatcherShutdownMillis = settings.DispatcherDefaultShutdown.toMillis
lazy val defaultGlobalDispatcher =
- AkkaConfig.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
+ settings.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
/**
* Creates an thread based dispatcher serving a single actor through the same single thread.
@@ -105,7 +105,7 @@ class Dispatchers(
* Has a fluent builder interface for configuring its semantics.
*/
def newDispatcher(name: String) =
- ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput,
+ ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, settings.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/**
@@ -132,7 +132,7 @@ class Dispatchers(
* Has a fluent builder interface for configuring its semantics.
*/
def newBalancingDispatcher(name: String) =
- ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput,
+ ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, settings.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/**
@@ -166,7 +166,7 @@ class Dispatchers(
* or else use the supplied default dispatcher
*/
def fromConfig(key: String, default: ⇒ MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher =
- AkkaConfig.config getSection key flatMap from getOrElse default
+ settings.config getSection key flatMap from getOrElse default
/*
* Creates of obtains a dispatcher from a ConfigMap according to the format below
@@ -193,8 +193,8 @@ class Dispatchers(
*/
def from(cfg: Configuration): Option[MessageDispatcher] = {
cfg.getString("type") flatMap {
- case "Dispatcher" ⇒ Some(new DispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler))
- case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler))
+ case "Dispatcher" ⇒ Some(new DispatcherConfigurator(settings, deadLetterMailbox, eventStream, scheduler))
+ case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(settings, deadLetterMailbox, eventStream, scheduler))
case "GlobalDispatcher" ⇒ None //TODO FIXME remove this
case fqn ⇒
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
@@ -214,26 +214,26 @@ class Dispatchers(
}
}
-class DispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) {
+class DispatcherConfigurator(settings: Settings, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(settings, eventStream) {
def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler,
config.getString("name", newUuid.toString),
- config.getInt("throughput", AkkaConfig.DispatcherThroughput),
- config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
+ config.getInt("throughput", settings.DispatcherThroughput),
+ config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
threadPoolConfig,
- AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
+ settings.DispatcherDefaultShutdown.toMillis)).build
}
}
-class BalancingDispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) {
+class BalancingDispatcherConfigurator(settings: Settings, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(settings, eventStream) {
def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler,
config.getString("name", newUuid.toString),
- config.getInt("throughput", AkkaConfig.DispatcherThroughput),
- config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
+ config.getInt("throughput", settings.DispatcherThroughput),
+ config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
threadPoolConfig,
- AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
+ settings.DispatcherDefaultShutdown.toMillis)).build
}
}
diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala
index 09d82e7c09..56b17399e3 100644
--- a/akka-actor/src/main/scala/akka/event/Logging.scala
+++ b/akka-actor/src/main/scala/akka/event/Logging.scala
@@ -5,7 +5,7 @@ package akka.event
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, ActorSystemImpl, simpleName }
import akka.AkkaException
-import akka.actor.ActorSystem.AkkaConfig
+import akka.actor.ActorSystem.Settings
import akka.util.ReflectiveAccess
import akka.config.ConfigurationException
import akka.util.ReentrantGuard
@@ -66,7 +66,7 @@ trait LoggingBus extends ActorEventBus {
_logLevel = level
}
- private[akka] def startStdoutLogger(config: AkkaConfig) {
+ private[akka] def startStdoutLogger(config: Settings) {
val level = levelFor(config.StdoutLogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.StdoutLogLevel))
ErrorLevel
@@ -80,12 +80,12 @@ trait LoggingBus extends ActorEventBus {
}
private[akka] def startDefaultLoggers(app: ActorSystemImpl) {
- val level = levelFor(app.AkkaConfig.LogLevel) getOrElse {
- StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.AkkaConfig.LogLevel))
+ val level = levelFor(app.settings.LogLevel) getOrElse {
+ StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.settings.LogLevel))
ErrorLevel
}
try {
- val defaultLoggers = app.AkkaConfig.EventHandlers match {
+ val defaultLoggers = app.settings.EventHandlers match {
case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil
case loggers ⇒ loggers
}
diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala
index 768be6dc9d..899e6790c7 100644
--- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala
+++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala
@@ -70,7 +70,7 @@ class Serialization(val app: ActorSystemImpl) {
* But "default" can be overridden in config
*/
val serializers: Map[String, Serializer] =
- app.AkkaConfig.config.getSection("akka.actor.serializers")
+ app.settings.config.getSection("akka.actor.serializers")
.map(_.map)
.getOrElse(Map())
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) {
@@ -81,7 +81,7 @@ class Serialization(val app: ActorSystemImpl) {
/**
* bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used
*/
- val bindings: Map[String, String] = app.AkkaConfig.config.getSection("akka.actor.serialization-bindings") map {
+ val bindings: Map[String, String] = app.settings.config.getSection("akka.actor.serialization-bindings") map {
_.map.foldLeft(Map[String, String]()) {
case (result, (k: String, vs: List[_])) ⇒ result ++ (vs collect { case v: String ⇒ (v, k) }) //All keys which are lists, take the Strings from them and Map them
case (result, _) ⇒ result //For any other values, just skip them, TODO: print out warnings?
diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala
index fbb27526f1..b9a1e86f33 100644
--- a/akka-actor/src/main/scala/akka/util/Duration.scala
+++ b/akka-actor/src/main/scala/akka/util/Duration.scala
@@ -278,7 +278,7 @@ abstract class Duration extends Serializable {
def /(other: Duration): Double
def unary_- : Duration
def finite_? : Boolean
- def dilated(implicit app: ActorSystem): Duration = this * app.AkkaConfig.TestTimeFactor
+ def dilated(implicit app: ActorSystem): Duration = this * app.settings.TestTimeFactor
def min(other: Duration): Duration = if (this < other) this else other
def max(other: Duration): Duration = if (this > other) this else other
def sleep(): Unit = Thread.sleep(toMillis)
diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala
index 316bbd5c95..8782a71720 100644
--- a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala
+++ b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala
@@ -27,8 +27,8 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
def this(app: ActorSystem) {
this(
- app.AkkaConfig.config.getInt("akka.remote.failure-detector.theshold", 8),
- app.AkkaConfig.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
+ app.settings.config.getInt("akka.remote.failure-detector.theshold", 8),
+ app.settings.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
}
private final val PhiFactor = 1.0 / math.log(10.0)
diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala
index 6d1461b43d..7c40ad428b 100644
--- a/akka-remote/src/main/scala/akka/remote/Remote.scala
+++ b/akka-remote/src/main/scala/akka/remote/Remote.scala
@@ -31,10 +31,10 @@ class Remote(val app: ActorSystemImpl, val nodename: String) {
val log = Logging(app, this)
import app._
- val AC = AkkaConfig
+ val AC = settings
import AC._
- // TODO move to AkkaConfig?
+ // TODO move to settings?
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
index cdb12d883c..0141b88445 100644
--- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
+++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
@@ -29,7 +29,7 @@ import akka.event.EventStream
* @author Jonas Bonér
*/
class RemoteActorRefProvider(
- val AkkaConfig: ActorSystem.AkkaConfig,
+ val settings: ActorSystem.Settings,
val rootPath: ActorPath,
val eventStream: EventStream,
val dispatcher: MessageDispatcher,
@@ -40,7 +40,7 @@ class RemoteActorRefProvider(
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
- val local = new LocalActorRefProvider(AkkaConfig, rootPath, eventStream, dispatcher, scheduler)
+ val local = new LocalActorRefProvider(settings, rootPath, eventStream, dispatcher, scheduler)
def deathWatch = local.deathWatch
def guardian = local.guardian
def systemGuardian = local.systemGuardian
@@ -67,7 +67,7 @@ class RemoteActorRefProvider(
private[akka] def deployer: Deployer = local.deployer
def defaultDispatcher = dispatcher
- def defaultTimeout = AkkaConfig.ActorTimeout
+ def defaultTimeout = settings.ActorTimeout
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(app, props, supervisor, supervisor.path / name, systemService)
diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala
index 996e38b44d..3632be6e8b 100644
--- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala
+++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala
@@ -351,8 +351,8 @@ class ActiveRemoteClientHandler(
*/
class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
- val serverSettings = new RemoteServerSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
- val clientSettings = new RemoteClientSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
+ val serverSettings = new RemoteServerSettings(app.settings.config, app.settings.DefaultTimeUnit)
+ val clientSettings = new RemoteClientSettings(app.settings.config, app.settings.DefaultTimeUnit)
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val clientsLock = new ReadWriteGuard
diff --git a/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala b/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala
index c346d94263..1d78b0c14b 100644
--- a/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala
+++ b/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala
@@ -19,14 +19,14 @@ class ConfiggyPropertyPlaceholderConfigurer extends PropertyPlaceholderConfigure
*/
override def setLocation(configgyResource: Resource) {
if (configgyResource eq null) throw new IllegalArgumentException("Property 'config' must be set")
- val properties = loadAkkaConfig(configgyResource)
+ val properties = loadSettings(configgyResource)
setProperties(properties)
}
/**
* Load the akka.conf and transform to properties.
*/
- private def loadAkkaConfig(configgyResource: Resource): Properties = {
+ private def loadSettings(configgyResource: Resource): Properties = {
val config = Configuration.fromFile(configgyResource.getFile.getPath)
val properties = new Properties()
config.map.foreach { case (k, v) ⇒ properties.put(k, v.asInstanceOf[AnyRef]); println("(k,v)=" + k + ", " + v) }
diff --git a/akka-spring/src/test/java/akka/spring/foo/IFoo.java b/akka-spring/src/test/java/akka/spring/foo/IFoo.java
index e47809f3af..0e5a294811 100644
--- a/akka-spring/src/test/java/akka/spring/foo/IFoo.java
+++ b/akka-spring/src/test/java/akka/spring/foo/IFoo.java
@@ -5,7 +5,7 @@ package akka.spring.foo;
* User: michaelkober
* Date: Aug 11, 2010
* Time: 12:49:58 PM
- * To change this template use File | Settings | File Templates.
+ * To change this template use File | settings | File Templates.
*/
public interface IFoo {
public String foo();
diff --git a/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java b/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java
index 825d797cf2..24e673a0c3 100644
--- a/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java
+++ b/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java
@@ -5,7 +5,7 @@ package akka.spring.foo;
* User: michaelkober
* Date: Aug 11, 2010
* Time: 12:01:00 PM
- * To change this template use File | Settings | File Templates.
+ * To change this template use File | settings | File Templates.
*/
public interface IMyPojo {
public void oneWay(String message);
diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala
index 2cbef7b719..6a31354e72 100644
--- a/akka-stm/src/main/scala/akka/agent/Agent.scala
+++ b/akka-stm/src/main/scala/akka/agent/Agent.scala
@@ -151,7 +151,7 @@ class Agent[T](initialValue: T, app: ActorSystem) {
def sendOff(f: T ⇒ T): Unit = {
send((value: T) ⇒ {
suspend()
- val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis)
+ val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.settings.ActorTimeoutMillis)
val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
threadBased ! Update(f)
value
@@ -169,7 +169,7 @@ class Agent[T](initialValue: T, app: ActorSystem) {
val result = new DefaultPromise[T](timeout)(app.dispatcher)
send((value: T) ⇒ {
suspend()
- val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis)
+ val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.settings.ActorTimeoutMillis)
val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]]
value
diff --git a/akka-stm/src/test/scala/config/ConfigSpec.scala b/akka-stm/src/test/scala/config/ConfigSpec.scala
index b6d963c7ea..ffb28e206e 100644
--- a/akka-stm/src/test/scala/config/ConfigSpec.scala
+++ b/akka-stm/src/test/scala/config/ConfigSpec.scala
@@ -16,7 +16,7 @@ class ConfigSpec extends WordSpec with MustMatchers {
"The default configuration file (i.e. akka-reference.conf)" should {
"contain all configuration properties for akka-stm that are used in code with their correct defaults" in {
- val config = ActorSystem("ConfigSpec").AkkaConfig.config
+ val config = ActorSystem("ConfigSpec").settings.config
import config._
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala
index 5b37d0573a..2f061f7eda 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala
@@ -33,7 +33,7 @@ class TestBarrier(count: Int) {
} catch {
case e: TimeoutException ⇒
throw new TestBarrierTimeoutException("Timeout of %s and time factor of %s"
- format (timeout.toString, app.AkkaConfig.TestTimeFactor))
+ format (timeout.toString, app.settings.TestTimeFactor))
}
}
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala
index 27f9574b43..7e1e946674 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala
@@ -83,9 +83,9 @@ abstract class EventFilter(occurrences: Int) {
app.eventStream publish TestEvent.Mute(this)
try {
val result = code
- if (!awaitDone(app.AkkaConfig.TestEventFilterLeeway))
+ if (!awaitDone(app.settings.TestEventFilterLeeway))
if (todo > 0)
- throw new AssertionError("Timeout (" + app.AkkaConfig.TestEventFilterLeeway + ") waiting for " + todo + " messages on " + this)
+ throw new AssertionError("Timeout (" + app.settings.TestEventFilterLeeway + ") waiting for " + todo + " messages on " + this)
else
throw new AssertionError("Received " + (-todo) + " messages too many on " + this)
result
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala
index 98154a58ac..a66ae79778 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala
@@ -125,9 +125,9 @@ class TestKit(_app: ActorSystem) {
/**
* Obtain time remaining for execution of the innermost enclosing `within`
* block or missing that it returns the properly dilated default for this
- * case from AkkaConfig (key "akka.test.single-expect-default").
+ * case from settings (key "akka.test.single-expect-default").
*/
- def remaining: Duration = if (end == Duration.Undefined) system.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now
+ def remaining: Duration = if (end == Duration.Undefined) system.settings.SingleExpectDefaultTimeout.dilated else end - now
/**
* Query queue status.
diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala
index f8a0cf3578..7bf5a59e1b 100644
--- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala
@@ -36,7 +36,7 @@ class TestLatch(count: Int = 1)(implicit app: ActorSystem) {
def await(timeout: Duration): Boolean = {
val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS)
if (!opened) throw new TestLatchTimeoutException(
- "Timeout of %s with time factor of %s" format (timeout.toString, app.AkkaConfig.TestTimeFactor))
+ "Timeout of %s with time factor of %s" format (timeout.toString, app.settings.TestTimeFactor))
opened
}
@@ -46,7 +46,7 @@ class TestLatch(count: Int = 1)(implicit app: ActorSystem) {
def awaitTimeout(timeout: Duration = TestLatch.DefaultTimeout) = {
val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS)
if (opened) throw new TestLatchNoTimeoutException(
- "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, app.AkkaConfig.TestTimeFactor))
+ "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, app.settings.TestTimeFactor))
opened
}
diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala
index 8a96b1b839..12096d2781 100644
--- a/akka-testkit/src/main/scala/akka/testkit/package.scala
+++ b/akka-testkit/src/main/scala/akka/testkit/package.scala
@@ -12,8 +12,8 @@ package object testkit {
try {
val result = block
- val stop = now + app.AkkaConfig.TestEventFilterLeeway.toMillis
- val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout (" + app.AkkaConfig.TestEventFilterLeeway + ") waiting for " + _)
+ val stop = now + app.settings.TestEventFilterLeeway.toMillis
+ val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout (" + app.settings.TestEventFilterLeeway + ") waiting for " + _)
if (failed.nonEmpty)
throw new AssertionError("Filter completion error:\n" + failed.mkString("\n"))
diff --git a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala
index 9a203e3b87..b799df3751 100644
--- a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala
+++ b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala
@@ -15,7 +15,7 @@ class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0)
val now = System.nanoTime
intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) }
val diff = System.nanoTime - now
- val target = (1000000000l * system.AkkaConfig.TestTimeFactor).toLong
+ val target = (1000000000l * system.settings.TestTimeFactor).toLong
diff must be > (target - 300000000l)
diff must be < (target + 300000000l)
}
diff --git a/project/sbt7/build/AkkaProject.scala b/project/sbt7/build/AkkaProject.scala
index 2a38c2ab22..7917edbefd 100644
--- a/project/sbt7/build/AkkaProject.scala
+++ b/project/sbt7/build/AkkaProject.scala
@@ -525,7 +525,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
class AkkaSbtPluginProject(info: ProjectInfo) extends PluginProject(info) {
val srcManagedScala = "src_managed" / "main" / "scala"
- lazy val addAkkaConfig = systemOptional[Boolean]("akka.release", false)
+ lazy val addSettings = systemOptional[Boolean]("akka.release", false)
lazy val generateAkkaSbtPlugin = {
val cleanSrcManaged = cleanTask(srcManagedScala) named ("clean src_managed")
@@ -533,7 +533,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
info.parent match {
case Some(project: ParentProject) =>
xsbt.FileUtilities.write((srcManagedScala / "AkkaProject.scala").asFile,
- GenerateAkkaSbtPlugin(project, addAkkaConfig.value))
+ GenerateAkkaSbtPlugin(project, addSettings.value))
case _ =>
}
None
@@ -550,9 +550,9 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec
}
object GenerateAkkaSbtPlugin {
- def apply(project: ParentProject, addAkkaConfig: Boolean): String = {
+ def apply(project: ParentProject, addSettings: Boolean): String = {
val extraConfigs = {
- if (addAkkaConfig) Set(ModuleConfiguration("se.scalablesolutions.akka", Repositories.AkkaRepo))
+ if (addSettings) Set(ModuleConfiguration("se.scalablesolutions.akka", Repositories.AkkaRepo))
else Set.empty[ModuleConfiguration]
}
val akkaModules = project.subProjects.values.map(_.name).flatMap{