rename AkkaConfig values to CamelCase

This commit is contained in:
Roland 2011-10-12 11:34:35 +02:00
parent 42e1bba164
commit 36ec202d94
30 changed files with 146 additions and 139 deletions

View file

@ -19,7 +19,7 @@ import akka.serialization.Serialization
object AkkaApplication {
val VERSION = "2.0-SNAPSHOT"
val Version = "2.0-SNAPSHOT"
val envHome = System.getenv("AKKA_HOME") match {
case null | "" | "." None
@ -31,7 +31,7 @@ object AkkaApplication {
case value Some(value)
}
val GLOBAL_HOME = systemHome orElse envHome
val GlobalHome = systemHome orElse envHome
val envConf = System.getenv("AKKA_MODE") match {
case null | "" None
@ -54,13 +54,15 @@ object AkkaApplication {
} catch { case _ None }
val fromHome = try {
Some(Configuration.fromFile(GLOBAL_HOME.get + "/config/" + defaultLocation))
Some(Configuration.fromFile(GlobalHome.get + "/config/" + defaultLocation))
} catch { case _ None }
val emptyConfig = Configuration.fromString("akka { version = \"" + VERSION + "\" }")
val emptyConfig = Configuration.fromString("akka { version = \"" + Version + "\" }")
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
def apply(name: String, config: Configuration) = new AkkaApplication(name, config)
def apply(name: String): AkkaApplication = new AkkaApplication(name)
def apply(): AkkaApplication = new AkkaApplication()
@ -76,36 +78,36 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
object AkkaConfig {
import config._
val CONFIG_VERSION = getString("akka.version", VERSION)
val TIME_UNIT = getString("akka.time-unit", "seconds")
val TIMEOUT = Timeout(Duration(getInt("akka.actor.timeout", 5), TIME_UNIT))
val TimeoutMillis = TIMEOUT.duration.toMillis
val SERIALIZE_MESSAGES = getBool("akka.actor.serialize-messages", false)
val ConfigVersion = getString("akka.version", Version)
val DefaultTimeUnit = getString("akka.time-unit", "seconds")
val ActorTimeout = Timeout(Duration(getInt("akka.actor.timeout", 5), DefaultTimeUnit))
val ActorTimeoutMillis = ActorTimeout.duration.toMillis
val SerializeAllMessages = getBool("akka.actor.serialize-messages", false)
val LogLevel = getString("akka.event-handler-level", "INFO")
val EventHandlers = getList("akka.event-handlers")
val ADD_LOGGING_RECEIVE = getBool("akka.actor.debug.receive", false)
val DEBUG_AUTO_RECEIVE = getBool("akka.actor.debug.autoreceive", false)
val DEBUG_LIFECYCLE = getBool("akka.actor.debug.lifecycle", false)
val AddLoggingReceive = getBool("akka.actor.debug.receive", false)
val DebugAutoReceive = getBool("akka.actor.debug.autoreceive", false)
val DebugLifecycle = getBool("akka.actor.debug.lifecycle", false)
val FsmDebugEvent = getBool("akka.actor.debug.fsm", false)
val DispatcherThroughput = getInt("akka.actor.throughput", 5)
val DispatcherDefaultShutdown = getLong("akka.actor.dispatcher-shutdown-timeout").
map(time Duration(time, TIME_UNIT)).
map(time Duration(time, DefaultTimeUnit)).
getOrElse(Duration(1000, TimeUnit.MILLISECONDS))
val MailboxCapacity = getInt("akka.actor.default-dispatcher.mailbox-capacity", -1)
val MailboxPushTimeout = Duration(getInt("akka.actor.default-dispatcher.mailbox-push-timeout-time", 10), TIME_UNIT)
val ThroughputDeadlineTime = Duration(getInt("akka.actor.throughput-deadline-time", -1), TIME_UNIT)
val MailboxPushTimeout = Duration(getInt("akka.actor.default-dispatcher.mailbox-push-timeout-time", 10), DefaultTimeUnit)
val DispatcherThroughputDeadlineTime = Duration(getInt("akka.actor.throughput-deadline-time", -1), DefaultTimeUnit)
val HOME = getString("akka.home")
val BOOT_CLASSES = getList("akka.boot")
val Home = getString("akka.home")
val BootClasses = getList("akka.boot")
val ENABLED_MODULES = getList("akka.enabled-modules")
val CLUSTER_ENABLED = ENABLED_MODULES exists (_ == "cluster")
val EnabledModules = getList("akka.enabled-modules")
val ClusterEnabled = EnabledModules exists (_ == "cluster")
val ClusterName = getString("akka.cluster.name", "default")
val REMOTE_TRANSPORT = getString("akka.remote.layer", "akka.remote.netty.NettyRemoteSupport")
val REMOTE_SERVER_PORT = getInt("akka.remote.server.port", 2552)
val RemoteTransport = getString("akka.remote.layer", "akka.remote.netty.NettyRemoteSupport")
val RemoteServerPort = getInt("akka.remote.server.port", 2552)
}
// Java API
@ -138,9 +140,9 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
case value value
}
if (CONFIG_VERSION != VERSION)
throw new ConfigurationException("Akka JAR version [" + VERSION +
"] does not match the provided config version [" + CONFIG_VERSION + "]")
if (ConfigVersion != Version)
throw new ConfigurationException("Akka JAR version [" + Version +
"] does not match the provided config version [" + ConfigVersion + "]")
// TODO correctly pull its config from the config
val dispatcherFactory = new Dispatchers(this)

View file

@ -139,7 +139,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 application: AkkaApplication) = application.AkkaConfig.TIMEOUT
implicit def defaultTimeout(implicit application: AkkaApplication) = application.AkkaConfig.ActorTimeout
}
object Actor {
@ -218,7 +218,7 @@ trait Actor {
/**
* The default timeout, based on the config setting 'akka.actor.timeout'
*/
implicit val defaultTimeout = config.TIMEOUT
implicit val defaultTimeout = config.ActorTimeout
/**
* Wrap a Receive partial function in a logging enclosure, which sends a
@ -234,7 +234,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 (config.ADD_LOGGING_RECEIVE) LoggingReceive(self, r) else r
def loggable(self: AnyRef)(r: Receive): Receive = if (config.AddLoggingReceive) LoggingReceive(self, r) else r
/**
* Some[ActorRef] representation of the 'self' ActorRef reference.
@ -423,7 +423,7 @@ trait Actor {
throw new InvalidMessageException("Message from [" + channel + "] to [" + self.toString + "] is null")
def autoReceiveMessage(msg: AutoReceivedMessage) {
if (config.DEBUG_AUTO_RECEIVE) EventHandler.debug(this, "received AutoReceiveMessage " + msg)
if (config.DebugAutoReceive) EventHandler.debug(this, "received AutoReceiveMessage " + msg)
msg match {
case HotSwap(code, discardOld) become(code(self), discardOld)

View file

@ -333,7 +333,7 @@ private[akka] class ActorCell(
actor = created
created.preStart()
checkReceiveTimeout
if (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(created, "started")
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(created, "started")
} catch {
case e try {
EventHandler.error(e, this, "error while creating actor")
@ -347,7 +347,7 @@ private[akka] class ActorCell(
def recreate(cause: Throwable): Unit = try {
val failedActor = actor
if (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(failedActor, "restarting")
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(failedActor, "restarting")
val freshActor = newActor()
if (failedActor ne null) {
val c = currentMessage //One read only plz
@ -361,7 +361,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 (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(freshActor, "restarted")
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(freshActor, "restarted")
dispatcher.resume(this) //FIXME should this be moved down?
@ -390,7 +390,7 @@ private[akka] class ActorCell(
try {
val a = actor
if (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(a, "stopping")
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(a, "stopping")
if (a ne null) a.postStop()
//Stop supervised actors
@ -416,7 +416,7 @@ private[akka] class ActorCell(
val links = _children
if (!links.exists(_.child == child)) {
_children = links :+ ChildRestartStats(child)
if (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(actor, "now supervising " + child)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "now supervising " + child)
} else EventHandler.warning(actor, "Already supervising " + child)
}
@ -428,10 +428,10 @@ private[akka] class ActorCell(
case Recreate(cause) recreate(cause)
case Link(subject)
akka.event.InVMMonitoring.link(self, subject)
if (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(actor, "now monitoring " + subject)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "now monitoring " + subject)
case Unlink(subject)
akka.event.InVMMonitoring.unlink(self, subject)
if (application.AkkaConfig.DEBUG_LIFECYCLE) EventHandler.debug(actor, "stopped monitoring " + subject)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "stopped monitoring " + subject)
case Suspend suspend()
case Resume resume()
case Terminate terminate()

View file

@ -17,12 +17,12 @@ trait BootableActorLoaderService extends Bootable {
def app: AkkaApplication
val BOOT_CLASSES = app.AkkaConfig.BOOT_CLASSES
val BOOT_CLASSES = app.AkkaConfig.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.AkkaConfig.Home.isDefined) {
val DEPLOY = app.AkkaConfig.Home.get + "/deploy"
val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) {
System.exit(-1)

View file

@ -222,7 +222,7 @@ class TypedActor(val application: AkkaApplication) {
//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 Timeout(Duration.MinusInf) application.AkkaConfig.TIMEOUT
case Timeout(Duration.MinusInf) application.AkkaConfig.ActorTimeout
case x x
}
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T]

View file

@ -322,7 +322,7 @@ abstract class MessageDispatcherConfigurator(val application: AkkaApplication) {
else {
val duration = Duration(
config.getInt("mailbox-push-timeout-time", application.AkkaConfig.MailboxPushTimeout.toMillis.toInt),
application.AkkaConfig.TIME_UNIT)
application.AkkaConfig.DefaultTimeUnit)
BoundedMailbox(capacity, duration)
}
}
@ -332,7 +332,7 @@ abstract class MessageDispatcherConfigurator(val application: AkkaApplication) {
//Apply the following options to the config if they are present in the config
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()).configure(
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, application.AkkaConfig.TIME_UNIT))),
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, application.AkkaConfig.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)),

View file

@ -44,14 +44,14 @@ import akka.AkkaApplication
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Dispatchers(val application: AkkaApplication) {
val THROUGHPUT_DEADLINE_TIME_MILLIS = application.AkkaConfig.ThroughputDeadlineTime.toMillis.toInt
val MAILBOX_TYPE: MailboxType =
val ThroughputDeadlineTimeMillis = application.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType =
if (application.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(application.AkkaConfig.MailboxCapacity, application.AkkaConfig.MailboxPushTimeout)
val DISPATCHER_SHUTDOWN_TIMEOUT = application.AkkaConfig.DispatcherDefaultShutdown.toMillis
val DispatcherShutdownMillis = application.AkkaConfig.DispatcherDefaultShutdown.toMillis
lazy val defaultGlobalDispatcher =
application.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MAILBOX_TYPE).build
application.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.
@ -60,8 +60,8 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(actor: LocalActorRef) = actor match {
case null new PinnedDispatcher(null, "anon", MAILBOX_TYPE, DISPATCHER_SHUTDOWN_TIMEOUT)
case some new PinnedDispatcher(some.underlying, some.underlying.uuid.toString, MAILBOX_TYPE, DISPATCHER_SHUTDOWN_TIMEOUT)
case null new PinnedDispatcher(null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(some.underlying, some.underlying.uuid.toString, MailboxType, DispatcherShutdownMillis)
}
/**
@ -71,8 +71,8 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
case null new PinnedDispatcher(null, "anon", mailboxType, DISPATCHER_SHUTDOWN_TIMEOUT)
case some new PinnedDispatcher(some.underlying, some.underlying.uuid.toString, mailboxType, DISPATCHER_SHUTDOWN_TIMEOUT)
case null new PinnedDispatcher(null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(some.underlying, some.underlying.uuid.toString, mailboxType, DispatcherShutdownMillis)
}
/**
@ -81,7 +81,7 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(name: String, mailboxType: MailboxType) =
new PinnedDispatcher(null, name, mailboxType, DISPATCHER_SHUTDOWN_TIMEOUT)
new PinnedDispatcher(null, name, mailboxType, DispatcherShutdownMillis)
/**
* Creates an thread based dispatcher serving a single actor through the same single thread.
@ -89,7 +89,7 @@ class Dispatchers(val application: AkkaApplication) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(name: String) =
new PinnedDispatcher(null, name, MAILBOX_TYPE, DISPATCHER_SHUTDOWN_TIMEOUT)
new PinnedDispatcher(null, name, MailboxType, DispatcherShutdownMillis)
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -98,7 +98,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(name, application.AkkaConfig.DispatcherThroughput,
THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -107,7 +107,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
new Dispatcher(name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -116,7 +116,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(name, throughput, throughputDeadlineMs, mailboxType, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
new Dispatcher(name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -125,7 +125,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(name, application.AkkaConfig.DispatcherThroughput,
THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -134,7 +134,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String, throughput: Int) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
new BalancingDispatcher(name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -143,7 +143,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
new BalancingDispatcher(name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -152,7 +152,7 @@ class Dispatchers(val application: AkkaApplication) {
*/
def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config, DISPATCHER_SHUTDOWN_TIMEOUT), ThreadPoolConfig())
new BalancingDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Utility function that tries to load the specified dispatcher config from the akka.conf
* or else use the supplied default dispatcher
@ -211,7 +211,7 @@ class DispatcherConfigurator(application: AkkaApplication) extends MessageDispat
configureThreadPool(config, threadPoolConfig new Dispatcher(
config.getString("name", newUuid.toString),
config.getInt("throughput", application.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", application.AkkaConfig.ThroughputDeadlineTime.toMillis.toInt),
config.getInt("throughput-deadline-time", application.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
threadPoolConfig,
application.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
@ -223,7 +223,7 @@ class BalancingDispatcherConfigurator(application: AkkaApplication) extends Mess
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(
config.getString("name", newUuid.toString),
config.getInt("throughput", application.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", application.AkkaConfig.ThroughputDeadlineTime.toMillis.toInt),
config.getInt("throughput-deadline-time", application.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
threadPoolConfig,
application.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build

View file

@ -333,10 +333,10 @@ trait RemoteServerModule extends RemoteModule { this: RemoteSupport ⇒
trait RemoteClientModule extends RemoteModule { self: RemoteSupport
def actorFor(address: String, hostname: String, port: Int): ActorRef =
actorFor(address, app.AkkaConfig.TimeoutMillis, hostname, port, None)
actorFor(address, app.AkkaConfig.ActorTimeoutMillis, hostname, port, None)
def actorFor(address: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
actorFor(address, app.AkkaConfig.TimeoutMillis, hostname, port, Some(loader))
actorFor(address, app.AkkaConfig.ActorTimeoutMillis, hostname, port, Some(loader))
def actorFor(address: String, timeout: Long, hostname: String, port: Int): ActorRef =
actorFor(address, timeout, hostname, port, None)

View file

@ -86,6 +86,6 @@ class AkkaLoader(application: AkkaApplication) {
==============================================================================
Running version %s
==============================================================================
""".format(AkkaApplication.VERSION))
""".format(AkkaApplication.Version))
}
}

View file

@ -133,7 +133,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ClusterModule {
lazy val isEnabled = application.AkkaConfig.CLUSTER_ENABLED //&& clusterInstance.isDefined
lazy val isEnabled = application.AkkaConfig.ClusterEnabled //&& clusterInstance.isDefined
lazy val clusterRefClass: Class[_] = getClassFor("akka.cluster.ClusterActorRef") match {
case Left(e) throw e
@ -239,9 +239,9 @@ class ReflectiveAccess(val application: AkkaApplication) {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RemoteModule {
val TRANSPORT = application.AkkaConfig.REMOTE_TRANSPORT
val TRANSPORT = application.AkkaConfig.RemoteTransport
val configDefaultAddress = new InetSocketAddress(application.hostname, application.AkkaConfig.REMOTE_SERVER_PORT)
val configDefaultAddress = new InetSocketAddress(application.hostname, application.AkkaConfig.RemoteServerPort)
lazy val isEnabled = remoteSupportClass.isDefined