rename AkkaConfig values to CamelCase
This commit is contained in:
parent
42e1bba164
commit
36ec202d94
30 changed files with 146 additions and 139 deletions
|
|
@ -20,7 +20,7 @@ import scala.Right;
|
|||
public class JavaFutureTests {
|
||||
|
||||
private final AkkaApplication app = new AkkaApplication();
|
||||
private final Timeout t = app.akkaConfig().TIMEOUT();
|
||||
private final Timeout t = app.akkaConfig().ActorTimeout();
|
||||
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);
|
||||
|
||||
@Test public void mustBeAbleToMapAFuture() {
|
||||
|
|
|
|||
|
|
@ -16,8 +16,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
|
|||
}
|
||||
}, timeout = t))
|
||||
|
||||
val defaultTimeout = app.AkkaConfig.TIMEOUT.duration
|
||||
val testTimeout = if (app.AkkaConfig.TIMEOUT.duration < 400.millis) 500 millis else 100 millis
|
||||
val defaultTimeout = app.AkkaConfig.ActorTimeout.duration
|
||||
val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
|
||||
|
||||
"An Actor-based Future" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -489,8 +489,8 @@ abstract class ActorModelSpec extends AkkaSpec {
|
|||
class DispatcherModelTest extends ActorModelSpec {
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher("foo", app.AkkaConfig.DispatcherThroughput,
|
||||
app.dispatcherFactory.THROUGHPUT_DEADLINE_TIME_MILLIS, app.dispatcherFactory.MAILBOX_TYPE,
|
||||
config, app.dispatcherFactory.DISPATCHER_SHUTDOWN_TIMEOUT) with MessageDispatcherInterceptor,
|
||||
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
|
||||
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
def dispatcherType = "Dispatcher"
|
||||
}
|
||||
|
|
@ -498,8 +498,8 @@ class DispatcherModelTest extends ActorModelSpec {
|
|||
class BalancingDispatcherModelTest extends ActorModelSpec {
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher("foo", 1, // TODO check why 1 here? (came from old test)
|
||||
app.dispatcherFactory.THROUGHPUT_DEADLINE_TIME_MILLIS, app.dispatcherFactory.MAILBOX_TYPE,
|
||||
config, app.dispatcherFactory.DISPATCHER_SHUTDOWN_TIMEOUT) with MessageDispatcherInterceptor,
|
||||
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
|
||||
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
def dispatcherType = "Balancing Dispatcher"
|
||||
|
||||
|
|
|
|||
|
|
@ -69,7 +69,7 @@ class DispatcherActorSpec extends AkkaSpec {
|
|||
|
||||
"respect the throughput setting" in {
|
||||
val throughputDispatcher = app.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MAILBOX_TYPE).
|
||||
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType).
|
||||
setCorePoolSize(1).
|
||||
build
|
||||
|
||||
|
|
@ -98,7 +98,7 @@ class DispatcherActorSpec extends AkkaSpec {
|
|||
"respect throughput deadline" in {
|
||||
val deadlineMs = 100
|
||||
val throughputDispatcher = app.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 2, deadlineMs, app.dispatcherFactory.MAILBOX_TYPE).
|
||||
newDispatcher("THROUGHPUT", 2, deadlineMs, app.dispatcherFactory.MailboxType).
|
||||
setCorePoolSize(1).
|
||||
build
|
||||
val works = new AtomicBoolean(true)
|
||||
|
|
|
|||
|
|
@ -186,7 +186,7 @@ class Report(app: AkkaApplication,
|
|||
sb.append("Args:\n ").append(args)
|
||||
sb.append("\n")
|
||||
|
||||
sb.append("Akka version: ").append(app.AkkaConfig.CONFIG_VERSION)
|
||||
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
|
||||
sb.append("\n")
|
||||
sb.append("Akka config:")
|
||||
for (key ← app.config.keys) {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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)),
|
||||
|
|
|
|||
|
|
@ -44,14 +44,14 @@ import akka.AkkaApplication
|
|||
* @author <a href="http://jonasboner.com">Jonas Boné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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -86,6 +86,6 @@ class AkkaLoader(application: AkkaApplication) {
|
|||
==============================================================================
|
||||
Running version %s
|
||||
==============================================================================
|
||||
""".format(AkkaApplication.VERSION))
|
||||
""".format(AkkaApplication.Version))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -133,7 +133,7 @@ class ReflectiveAccess(val application: AkkaApplication) {
|
|||
* @author <a href="http://jonasboner.com">Jonas Boné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é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
|
||||
|
||||
|
|
|
|||
|
|
@ -11,24 +11,24 @@ import akka.AkkaApplication
|
|||
class RemoteClientSettings(val app: AkkaApplication) {
|
||||
|
||||
import app.config
|
||||
import app.AkkaConfig.TIME_UNIT
|
||||
import app.AkkaConfig.DefaultTimeUnit
|
||||
|
||||
val SECURE_COOKIE: Option[String] = config.getString("akka.remote.secure-cookie", "") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
|
||||
val RECONNECTION_TIME_WINDOW = Duration(config.getInt("akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
|
||||
val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 3600), TIME_UNIT)
|
||||
val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT)
|
||||
val REAP_FUTURES_DELAY = Duration(config.getInt("akka.remote.client.reap-futures-delay", 5), TIME_UNIT)
|
||||
val RECONNECTION_TIME_WINDOW = Duration(config.getInt("akka.remote.client.reconnection-time-window", 600), DefaultTimeUnit).toMillis
|
||||
val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 3600), DefaultTimeUnit)
|
||||
val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), DefaultTimeUnit)
|
||||
val REAP_FUTURES_DELAY = Duration(config.getInt("akka.remote.client.reap-futures-delay", 5), DefaultTimeUnit)
|
||||
val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.client.message-frame-size", 1048576)
|
||||
}
|
||||
|
||||
class RemoteServerSettings(val app: AkkaApplication) {
|
||||
|
||||
import app.config
|
||||
import app.AkkaConfig.TIME_UNIT
|
||||
import app.AkkaConfig.DefaultTimeUnit
|
||||
|
||||
val isRemotingEnabled = config.getList("akka.enabled-modules").exists(_ == "cluster")
|
||||
val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.server.message-frame-size", 1048576)
|
||||
|
|
@ -42,11 +42,11 @@ class RemoteServerSettings(val app: AkkaApplication) {
|
|||
|
||||
val UNTRUSTED_MODE = config.getBool("akka.remote.server.untrusted-mode", false)
|
||||
val PORT = config.getInt("akka.remote.server.port", 2552)
|
||||
val CONNECTION_TIMEOUT = Duration(config.getInt("akka.remote.server.connection-timeout", 100), TIME_UNIT)
|
||||
val CONNECTION_TIMEOUT = Duration(config.getInt("akka.remote.server.connection-timeout", 100), DefaultTimeUnit)
|
||||
|
||||
val BACKLOG = config.getInt("akka.remote.server.backlog", 4096)
|
||||
|
||||
val EXECUTION_POOL_KEEPALIVE = Duration(config.getInt("akka.remote.server.execution-pool-keepalive", 60), TIME_UNIT)
|
||||
val EXECUTION_POOL_KEEPALIVE = Duration(config.getInt("akka.remote.server.execution-pool-keepalive", 60), DefaultTimeUnit)
|
||||
|
||||
val EXECUTION_POOL_SIZE = {
|
||||
val sz = config.getInt("akka.remote.server.execution-pool-size", 16)
|
||||
|
|
|
|||
|
|
@ -28,13 +28,13 @@ import com.eaio.uuid.UUID
|
|||
class Remote(val app: AkkaApplication) extends RemoteService {
|
||||
|
||||
import app.config
|
||||
import app.AkkaConfig.TIME_UNIT
|
||||
import app.AkkaConfig.DefaultTimeUnit
|
||||
|
||||
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
|
||||
val remoteDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), TIME_UNIT).toMillis.toInt
|
||||
val remoteDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt
|
||||
|
||||
val hostname = app.hostname
|
||||
val port = app.AkkaConfig.REMOTE_SERVER_PORT
|
||||
val port = app.AkkaConfig.RemoteServerPort
|
||||
|
||||
val remoteDaemonServiceName = "akka-remote-daemon".intern
|
||||
|
||||
|
|
|
|||
|
|
@ -197,7 +197,7 @@ abstract class RemoteClient private[akka] (
|
|||
isOneWay: Boolean,
|
||||
actorRef: ActorRef): Option[Promise[T]] = {
|
||||
val messageProtocol = serialization.createRemoteMessageProtocolBuilder(
|
||||
Some(actorRef), Left(actorRef.uuid), actorRef.address, app.AkkaConfig.TimeoutMillis, Right(message), isOneWay, senderOption).build
|
||||
Some(actorRef), Left(actorRef.uuid), actorRef.address, app.AkkaConfig.ActorTimeoutMillis, Right(message), isOneWay, senderOption).build
|
||||
send(messageProtocol, senderFuture)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -70,7 +70,7 @@ class ActorSerialization(val app: AkkaApplication) {
|
|||
val builder = SerializedActorRefProtocol.newBuilder
|
||||
.setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build)
|
||||
.setAddress(actorRef.address)
|
||||
.setTimeout(app.AkkaConfig.TimeoutMillis)
|
||||
.setTimeout(app.AkkaConfig.ActorTimeoutMillis)
|
||||
|
||||
replicationScheme match {
|
||||
case _: Transient | Transient ⇒
|
||||
|
|
@ -104,7 +104,7 @@ class ActorSerialization(val app: AkkaApplication) {
|
|||
Option(m.receiver.ref),
|
||||
Left(actorRef.uuid),
|
||||
actorRef.address,
|
||||
app.AkkaConfig.TimeoutMillis,
|
||||
app.AkkaConfig.ActorTimeoutMillis,
|
||||
Right(m.message),
|
||||
false,
|
||||
m.channel match {
|
||||
|
|
@ -201,7 +201,7 @@ class ActorSerialization(val app: AkkaApplication) {
|
|||
}
|
||||
|
||||
val props = Props(creator = factory,
|
||||
timeout = if (protocol.hasTimeout) protocol.getTimeout else app.AkkaConfig.TIMEOUT,
|
||||
timeout = if (protocol.hasTimeout) protocol.getTimeout else app.AkkaConfig.ActorTimeout,
|
||||
supervisor = storedSupervisor //TODO what dispatcher should it use?
|
||||
//TODO what faultHandler should it use?
|
||||
//
|
||||
|
|
@ -272,7 +272,7 @@ class RemoteActorSerialization(val app: AkkaApplication) {
|
|||
RemoteActorRefProtocol.newBuilder
|
||||
.setInetSocketAddress(ByteString.copyFrom(JavaSerializer.toBinary(remoteAddress)))
|
||||
.setAddress(actor.address)
|
||||
.setTimeout(app.AkkaConfig.TimeoutMillis)
|
||||
.setTimeout(app.AkkaConfig.ActorTimeoutMillis)
|
||||
.build
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -153,7 +153,7 @@ class Agent[T](initialValue: T, application: AkkaApplication) {
|
|||
def sendOff(f: T ⇒ T): Unit = {
|
||||
send((value: T) ⇒ {
|
||||
suspend()
|
||||
val pinnedDispatcher = new PinnedDispatcher(null, "agent-send-off", UnboundedMailbox(), application.AkkaConfig.TimeoutMillis)
|
||||
val pinnedDispatcher = new PinnedDispatcher(null, "agent-send-off", UnboundedMailbox(), application.AkkaConfig.ActorTimeoutMillis)
|
||||
val threadBased = application.createActor(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
|
||||
threadBased ! Update(f)
|
||||
value
|
||||
|
|
@ -171,7 +171,7 @@ class Agent[T](initialValue: T, application: AkkaApplication) {
|
|||
val result = new DefaultPromise[T](timeout)(application.dispatcher)
|
||||
send((value: T) ⇒ {
|
||||
suspend()
|
||||
val pinnedDispatcher = new PinnedDispatcher(null, "agent-alter-off", UnboundedMailbox(), application.AkkaConfig.TimeoutMillis)
|
||||
val pinnedDispatcher = new PinnedDispatcher(null, "agent-alter-off", UnboundedMailbox(), application.AkkaConfig.ActorTimeoutMillis)
|
||||
val threadBased = application.createActor(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
|
||||
result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]]
|
||||
value
|
||||
|
|
|
|||
|
|
@ -4,11 +4,11 @@
|
|||
|
||||
package akka.tutorial.java.second;
|
||||
|
||||
import static akka.actor.Actors.actorOf;
|
||||
import static akka.actor.Actors.poisonPill;
|
||||
import static java.lang.System.currentTimeMillis;
|
||||
import static java.util.Arrays.asList;
|
||||
|
||||
import akka.AkkaApplication;
|
||||
import akka.routing.RoutedProps;
|
||||
import akka.routing.Routing;
|
||||
import scala.Option;
|
||||
|
|
@ -25,6 +25,8 @@ import java.util.LinkedList;
|
|||
|
||||
public class Pi {
|
||||
|
||||
private static final AkkaApplication app = new AkkaApplication();
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Pi pi = new Pi();
|
||||
pi.calculate(4, 10000, 10000);
|
||||
|
|
@ -99,11 +101,11 @@ public class Pi {
|
|||
|
||||
LinkedList<ActorRef> workers = new LinkedList<ActorRef>();
|
||||
for (int i = 0; i < nrOfWorkers; i++) {
|
||||
ActorRef worker = actorOf(Worker.class, "worker");
|
||||
ActorRef worker = app.createActor(Worker.class);
|
||||
workers.add(worker);
|
||||
}
|
||||
|
||||
router = Routing.actorOf(RoutedProps.apply().withConnections(workers).withRoundRobinRouter(), "pi");
|
||||
router = app.routing().actorOf(RoutedProps.apply().withConnections(workers).withRoundRobinRouter(), "pi");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -159,11 +161,11 @@ public class Pi {
|
|||
public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages) throws Exception {
|
||||
|
||||
// create the master
|
||||
ActorRef master = actorOf(new UntypedActorFactory() {
|
||||
ActorRef master = app.createActor(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements);
|
||||
}
|
||||
}, "worker");
|
||||
});
|
||||
|
||||
// start the calculation
|
||||
long start = currentTimeMillis();
|
||||
|
|
|
|||
|
|
@ -10,9 +10,12 @@ import System.{ currentTimeMillis ⇒ now }
|
|||
import akka.routing.Routing.Broadcast
|
||||
import akka.actor.{ Timeout, Channel, Actor, PoisonPill }
|
||||
import akka.routing.{ RoutedProps, Routing }
|
||||
import akka.AkkaApplication
|
||||
|
||||
object Pi extends App {
|
||||
|
||||
val app = AkkaApplication()
|
||||
|
||||
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
|
||||
|
||||
// ====================
|
||||
|
|
@ -50,10 +53,10 @@ object Pi extends App {
|
|||
var nrOfResults: Int = _
|
||||
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
|
||||
val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker])
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
val router = Routing.actorOf(RoutedProps().withConnections(workers).withRoundRobinRouter, "pi")
|
||||
val router = app.routing.actorOf(RoutedProps().withConnections(workers).withRoundRobinRouter, "pi")
|
||||
|
||||
// phase 1, can accept a Calculate message
|
||||
def scatter: Receive = {
|
||||
|
|
@ -96,7 +99,7 @@ object Pi extends App {
|
|||
// ==================
|
||||
def calculate(nrOfWorkers: Int, nrOfElements: Int, nrOfMessages: Int) {
|
||||
// create the master
|
||||
val master = actorOf(new Master(nrOfWorkers, nrOfElements, nrOfMessages))
|
||||
val master = app.createActor(new Master(nrOfWorkers, nrOfElements, nrOfMessages))
|
||||
|
||||
//start the calculation
|
||||
val start = now
|
||||
|
|
|
|||
|
|
@ -65,7 +65,7 @@ object AkkaBuild extends Build {
|
|||
lazy val stm = Project(
|
||||
id = "akka-stm",
|
||||
base = file("akka-stm"),
|
||||
dependencies = Seq(actor, testkit % "test"),
|
||||
dependencies = Seq(actor, testkit % "test->test"),
|
||||
settings = defaultSettings ++ Seq(
|
||||
libraryDependencies ++= Dependencies.stm
|
||||
)
|
||||
|
|
@ -74,7 +74,7 @@ object AkkaBuild extends Build {
|
|||
lazy val remote = Project(
|
||||
id = "akka-remote",
|
||||
base = file("akka-remote"),
|
||||
dependencies = Seq(stm, actorTests % "test->test", testkit % "test"),
|
||||
dependencies = Seq(stm, actorTests % "test->test", testkit % "test->test"),
|
||||
settings = defaultSettings ++ multiJvmSettings ++ Seq(
|
||||
libraryDependencies ++= Dependencies.cluster,
|
||||
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
|
||||
|
|
@ -108,7 +108,7 @@ object AkkaBuild extends Build {
|
|||
lazy val http = Project(
|
||||
id = "akka-http",
|
||||
base = file("akka-http"),
|
||||
dependencies = Seq(actor, testkit % "test"),
|
||||
dependencies = Seq(actor, testkit % "test->test"),
|
||||
settings = defaultSettings ++ Seq(
|
||||
libraryDependencies ++= Dependencies.http
|
||||
)
|
||||
|
|
@ -117,7 +117,7 @@ object AkkaBuild extends Build {
|
|||
lazy val slf4j = Project(
|
||||
id = "akka-slf4j",
|
||||
base = file("akka-slf4j"),
|
||||
dependencies = Seq(actor, testkit % "test"),
|
||||
dependencies = Seq(actor, testkit % "test->test"),
|
||||
settings = defaultSettings ++ Seq(
|
||||
libraryDependencies ++= Dependencies.slf4j
|
||||
)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue