rename akka.AkkaApplication to akka.actor.ActorSystem

Renaming it to System did not appeal after seeing that such a thing is
already imported from Predef ...
This commit is contained in:
Roland 2011-11-10 20:08:00 +01:00
parent c6e44ffef7
commit 945b1aedf9
79 changed files with 209 additions and 230 deletions

View file

@ -1,6 +1,6 @@
package akka.actor; package akka.actor;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import akka.japi.Creator; import akka.japi.Creator;
import org.junit.Test; import org.junit.Test;
import akka.actor.Actors; import akka.actor.Actors;
@ -9,7 +9,7 @@ import static org.junit.Assert.*;
public class JavaAPI { public class JavaAPI {
private AkkaApplication app = new AkkaApplication(); private ActorSystem app = new ActorSystem();
@Test void mustBeAbleToCreateActorRefFromClass() { @Test void mustBeAbleToCreateActorRefFromClass() {
ActorRef ref = app.actorOf(JavaAPITestActor.class); ActorRef ref = app.actorOf(JavaAPITestActor.class);

View file

@ -1,7 +1,7 @@
package akka.dispatch; package akka.dispatch;
import akka.actor.Timeout; import akka.actor.Timeout;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
@ -19,7 +19,7 @@ import scala.Right;
public class JavaFutureTests { public class JavaFutureTests {
private final AkkaApplication app = new AkkaApplication(); private final ActorSystem app = new ActorSystem();
private final Timeout t = app.AkkaConfig().ActorTimeout(); private final Timeout t = app.AkkaConfig().ActorTimeout();
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t); private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);

View file

@ -274,7 +274,7 @@ class ActorRefSpec extends AkkaSpec {
(intercept[java.lang.IllegalStateException] { (intercept[java.lang.IllegalStateException] {
in.readObject in.readObject
}).getMessage must be === "Trying to deserialize a serialized ActorRef without an AkkaApplication in scope." + }).getMessage must be === "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }" " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }"
} }

View file

@ -5,8 +5,6 @@
*/ */
package akka.actor package akka.actor
import akka.AkkaApplication
object Chameneos { object Chameneos {
sealed trait ChameneosEvent sealed trait ChameneosEvent
@ -109,7 +107,7 @@ object Chameneos {
def run { def run {
// System.setProperty("akka.config", "akka.conf") // System.setProperty("akka.config", "akka.conf")
Chameneos.start = System.currentTimeMillis Chameneos.start = System.currentTimeMillis
AkkaApplication().actorOf(new Mall(1000000, 4)) ActorSystem().actorOf(new Mall(1000000, 4))
Thread.sleep(10000) Thread.sleep(10000)
println("Elapsed: " + (end - start)) println("Elapsed: " + (end - start))
} }

View file

@ -12,13 +12,12 @@ import FSM._
import akka.util.Duration import akka.util.Duration
import akka.util.duration._ import akka.util.duration._
import akka.event._ import akka.event._
import akka.AkkaApplication import akka.actor.ActorSystem.defaultConfig
import akka.AkkaApplication.defaultConfig
import akka.config.Configuration import akka.config.Configuration
object FSMActorSpec { object FSMActorSpec {
class Latches(implicit app: AkkaApplication) { class Latches(implicit app: ActorSystem) {
val unlockedLatch = TestLatch() val unlockedLatch = TestLatch()
val lockedLatch = TestLatch() val lockedLatch = TestLatch()
val unhandledLatch = TestLatch() val unhandledLatch = TestLatch()
@ -194,7 +193,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
} }
"log events and transitions if asked to do so" in { "log events and transitions if asked to do so" in {
new TestKit(AkkaApplication("fsm event", AkkaApplication.defaultConfig ++ new TestKit(ActorSystem("fsm event", ActorSystem.defaultConfig ++
Configuration("akka.loglevel" -> "DEBUG", Configuration("akka.loglevel" -> "DEBUG",
"akka.actor.debug.fsm" -> true))) { "akka.actor.debug.fsm" -> true))) {
EventFilter.debug() intercept { EventFilter.debug() intercept {

View file

@ -8,12 +8,11 @@ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import Actor._ import Actor._
import akka.util.Duration import akka.util.Duration
import akka.AkkaApplication
object ForwardActorSpec { object ForwardActorSpec {
val ExpectedMessage = "FOO" val ExpectedMessage = "FOO"
def createForwardingChain(app: AkkaApplication): ActorRef = { def createForwardingChain(app: ActorSystem): ActorRef = {
val replier = app.actorOf(new Actor { val replier = app.actorOf(new Actor {
def receive = { case x sender ! x } def receive = { case x sender ! x }
}) })

View file

@ -7,8 +7,7 @@ import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.util.duration._ import akka.util.duration._
import akka.testkit._ import akka.testkit._
import org.scalatest.WordSpec import org.scalatest.WordSpec
import akka.AkkaApplication import akka.actor.ActorSystem.defaultConfig
import akka.AkkaApplication.defaultConfig
import akka.config.Configuration import akka.config.Configuration
import akka.event.Logging import akka.event.Logging
import akka.util.Duration import akka.util.Duration
@ -25,9 +24,9 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
import LoggingReceiveSpec._ import LoggingReceiveSpec._
val config = defaultConfig ++ Configuration("akka.event-handler-level" -> "DEBUG") val config = defaultConfig ++ Configuration("akka.event-handler-level" -> "DEBUG")
val appLogging = AkkaApplication("logging", config ++ Configuration("akka.actor.debug.receive" -> true)) val appLogging = ActorSystem("logging", config ++ Configuration("akka.actor.debug.receive" -> true))
val appAuto = AkkaApplication("autoreceive", config ++ Configuration("akka.actor.debug.autoreceive" -> true)) val appAuto = ActorSystem("autoreceive", config ++ Configuration("akka.actor.debug.autoreceive" -> true))
val appLifecycle = AkkaApplication("lifecycle", config ++ Configuration("akka.actor.debug.lifecycle" -> true)) val appLifecycle = ActorSystem("lifecycle", config ++ Configuration("akka.actor.debug.lifecycle" -> true))
val filter = TestEvent.Mute(EventFilter.custom { val filter = TestEvent.Mute(EventFilter.custom {
case _: Logging.Debug true case _: Logging.Debug true

View file

@ -13,7 +13,7 @@ import java.rmi.RemoteException
import org.junit.{ After, Test } import org.junit.{ After, Test }
import akka.actor._ import akka.actor._
import util.control.NoStackTrace import util.control.NoStackTrace
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.util.duration._ import akka.util.duration._
import akka.event.Logging.Error import akka.event.Logging.Error
@ -148,7 +148,7 @@ object ActorModelSpec {
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)( def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
starts: Long = dispatcher.starts.get(), starts: Long = dispatcher.starts.get(),
stops: Long = dispatcher.stops.get())(implicit app: AkkaApplication) { stops: Long = dispatcher.stops.get())(implicit app: ActorSystem) {
val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5 val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
try { try {
await(deadline)(starts == dispatcher.starts.get) await(deadline)(starts == dispatcher.starts.get)
@ -181,7 +181,7 @@ object ActorModelSpec {
unregisters: Long = 0, unregisters: Long = 0,
msgsReceived: Long = 0, msgsReceived: Long = 0,
msgsProcessed: Long = 0, msgsProcessed: Long = 0,
restarts: Long = 0)(implicit app: AkkaApplication) { restarts: Long = 0)(implicit app: ActorSystem) {
assertRef(actorRef, dispatcher)( assertRef(actorRef, dispatcher)(
suspensions, suspensions,
resumes, resumes,
@ -199,7 +199,7 @@ object ActorModelSpec {
unregisters: Long = statsFor(actorRef).unregisters.get(), unregisters: Long = statsFor(actorRef).unregisters.get(),
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(), msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(), msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: AkkaApplication) { restarts: Long = statsFor(actorRef).restarts.get())(implicit app: ActorSystem) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher)) val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val deadline = System.currentTimeMillis + 1000 val deadline = System.currentTimeMillis + 1000
try { try {

View file

@ -5,10 +5,10 @@
package akka.config package akka.config
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.AkkaApplication import akka.actor.ActorSystem
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfigSpec extends AkkaSpec(AkkaApplication("ConfigSpec", Configuration.fromFile("config/akka-reference.conf"))) { class ConfigSpec extends AkkaSpec(ActorSystem("ConfigSpec", Configuration.fromFile("config/akka-reference.conf"))) {
"The default configuration file (i.e. akka-reference.conf)" must { "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 { "contain all configuration properties for akka-actor that are used in code with their correct defaults" in {

View file

@ -5,7 +5,7 @@ import akka.performance.trading.domain.OrderbookRepository
import akka.actor.Actor._ import akka.actor.Actor._
import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher
import akka.actor.{ Props, ActorRef, PoisonPill } import akka.actor.{ Props, ActorRef, PoisonPill }
import akka.AkkaApplication import akka.actor.ActorSystem
trait TradingSystem { trait TradingSystem {
type ME type ME
@ -34,7 +34,7 @@ trait TradingSystem {
case class MatchingEngineInfo(primary: ME, standby: Option[ME], orderbooks: List[Orderbook]) case class MatchingEngineInfo(primary: ME, standby: Option[ME], orderbooks: List[Orderbook])
} }
class AkkaTradingSystem(val app: AkkaApplication) extends TradingSystem { class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
type ME = ActorRef type ME = ActorRef
type OR = ActorRef type OR = ActorRef

View file

@ -12,7 +12,7 @@ import java.io.PrintWriter
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
import scala.collection.mutable.{ Map MutableMap } import scala.collection.mutable.{ Map MutableMap }
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.event.Logging import akka.event.Logging
trait BenchResultRepository { trait BenchResultRepository {

View file

@ -7,11 +7,11 @@ import org.scalatest.BeforeAndAfterEach
import akka.actor.simpleName import akka.actor.simpleName
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.AkkaApplication import akka.actor.ActorSystem
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach { trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
def app: AkkaApplication def app: ActorSystem
def isBenchmark() = System.getProperty("benchmark") == "true" def isBenchmark() = System.getProperty("benchmark") == "true"

View file

@ -5,12 +5,12 @@ import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
import scala.collection.JavaConversions.asScalaBuffer import scala.collection.JavaConversions.asScalaBuffer
import scala.collection.JavaConversions.enumerationAsScalaIterator import scala.collection.JavaConversions.enumerationAsScalaIterator
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.event.Logging import akka.event.Logging
import scala.collection.immutable.TreeMap import scala.collection.immutable.TreeMap
class Report( class Report(
app: AkkaApplication, app: ActorSystem,
resultRepository: BenchResultRepository, resultRepository: BenchResultRepository,
compareResultWith: Option[String] = None) { compareResultWith: Option[String] = None) {

View file

@ -7,7 +7,7 @@ package akka.serialization
import akka.serialization.Serialization._ import akka.serialization.Serialization._
import scala.reflect._ import scala.reflect._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.AkkaApplication import akka.actor.ActorSystem
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import akka.actor.DeadLetterActorRef import akka.actor.DeadLetterActorRef
@ -68,7 +68,7 @@ class SerializeSpec extends AkkaSpec {
"serialize DeadLetterActorRef" in { "serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream() val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf) val out = new ObjectOutputStream(outbuf)
val a = new AkkaApplication() val a = new ActorSystem()
out.writeObject(a.deadLetters) out.writeObject(a.deadLetters)
out.flush() out.flush()
out.close() out.close()

View file

@ -15,7 +15,7 @@ import akka.japi.{ Creator, Procedure }
import akka.serialization.{ Serializer, Serialization } import akka.serialization.{ Serializer, Serialization }
import akka.event.Logging.Debug import akka.event.Logging.Debug
import akka.experimental import akka.experimental
import akka.{ AkkaApplication, AkkaException } import akka.AkkaException
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
@ -151,7 +151,7 @@ object Timeout {
implicit def durationToTimeout(duration: Duration) = new Timeout(duration) implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
implicit def intToTimeout(timeout: Int) = new Timeout(timeout) implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
implicit def longToTimeout(timeout: Long) = new Timeout(timeout) implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
implicit def defaultTimeout(implicit app: AkkaApplication) = app.AkkaConfig.ActorTimeout implicit def defaultTimeout(implicit app: ActorSystem) = app.AkkaConfig.ActorTimeout
} }
trait ActorLogging { this: Actor trait ActorLogging { this: Actor
@ -165,7 +165,7 @@ object Actor {
/** /**
* This decorator adds invocation logging to a Receive function. * This decorator adds invocation logging to a Receive function.
*/ */
class LoggingReceive(source: AnyRef, r: Receive)(implicit app: AkkaApplication) extends Receive { class LoggingReceive(source: AnyRef, r: Receive)(implicit app: ActorSystem) extends Receive {
def isDefinedAt(o: Any) = { def isDefinedAt(o: Any) = {
val handled = r.isDefinedAt(o) val handled = r.isDefinedAt(o)
app.mainbus.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o)) app.mainbus.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
@ -175,7 +175,7 @@ object Actor {
} }
object LoggingReceive { object LoggingReceive {
def apply(source: AnyRef, r: Receive)(implicit app: AkkaApplication): Receive = r match { def apply(source: AnyRef, r: Receive)(implicit app: ActorSystem): Receive = r match {
case _: LoggingReceive r case _: LoggingReceive r
case _ new LoggingReceive(source, r) case _ new LoggingReceive(source, r)
} }

View file

@ -8,7 +8,6 @@ import akka.dispatch._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable.{ Stack, TreeMap } import scala.collection.immutable.{ Stack, TreeMap }
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.AkkaApplication
import akka.event.Logging.{ Debug, Warning, Error } import akka.event.Logging.{ Debug, Warning, Error }
/** /**
@ -44,7 +43,7 @@ trait ActorContext extends ActorRefFactory with TypedActorFactory {
def handleChildTerminated(child: ActorRef): Unit def handleChildTerminated(child: ActorRef): Unit
def app: AkkaApplication def app: ActorSystem
def parent: ActorRef def parent: ActorRef
} }
@ -62,7 +61,7 @@ private[akka] object ActorCell {
//vars don't need volatile since it's protected with the mailbox status //vars don't need volatile since it's protected with the mailbox status
//Make sure that they are not read/written outside of a message processing (systemInvoke/invoke) //Make sure that they are not read/written outside of a message processing (systemInvoke/invoke)
private[akka] class ActorCell( private[akka] class ActorCell(
val app: AkkaApplication, val app: ActorSystem,
val self: ActorRef with ScalaActorRef, val self: ActorRef with ScalaActorRef,
val props: Props, val props: Props,
val parent: ActorRef, val parent: ActorRef,

View file

@ -4,8 +4,6 @@
package akka.actor package akka.actor
import akka.AkkaApplication
object ActorPath { object ActorPath {
final val separator = "/" final val separator = "/"
@ -14,13 +12,13 @@ object ActorPath {
/** /**
* Create an actor path from a string. * Create an actor path from a string.
*/ */
def apply(app: AkkaApplication, path: String): ActorPath = def apply(app: ActorSystem, path: String): ActorPath =
apply(app, split(path)) apply(app, split(path))
/** /**
* Create an actor path from an iterable. * Create an actor path from an iterable.
*/ */
def apply(app: AkkaApplication, path: Iterable[String]): ActorPath = def apply(app: ActorSystem, path: Iterable[String]): ActorPath =
path.foldLeft(app.root)(_ / _) path.foldLeft(app.root)(_ / _)
/** /**
@ -62,7 +60,7 @@ trait ActorPath {
/** /**
* The akka application for this path. * The akka application for this path.
*/ */
def app: AkkaApplication def app: ActorSystem
/** /**
* The name of the actor that this path refers to. * The name of the actor that this path refers to.
@ -100,7 +98,7 @@ trait ActorPath {
def isRoot: Boolean def isRoot: Boolean
} }
class RootActorPath(val app: AkkaApplication) extends ActorPath { class RootActorPath(val app: ActorSystem) extends ActorPath {
def name: String = "/" def name: String = "/"
@ -119,7 +117,7 @@ class RootActorPath(val app: AkkaApplication) extends ActorPath {
override def toString = ActorPath.separator override def toString = ActorPath.separator
} }
class ChildActorPath(val app: AkkaApplication, val parent: ActorPath, val name: String) extends ActorPath { class ChildActorPath(val app: ActorSystem, val parent: ActorPath, val name: String) extends ActorPath {
def /(child: String): ActorPath = new ChildActorPath(app, this, child) def /(child: String): ActorPath = new ChildActorPath(app, this, child)

View file

@ -8,7 +8,6 @@ import akka.dispatch._
import akka.util._ import akka.util._
import scala.collection.immutable.Stack import scala.collection.immutable.Stack
import java.lang.{ UnsupportedOperationException, IllegalStateException } import java.lang.{ UnsupportedOperationException, IllegalStateException }
import akka.AkkaApplication
import akka.serialization.Serialization import akka.serialization.Serialization
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.remote.RemoteAddress import akka.remote.RemoteAddress
@ -161,7 +160,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class LocalActorRef private[akka] ( class LocalActorRef private[akka] (
_app: AkkaApplication, _app: ActorSystem,
props: Props, props: Props,
_supervisor: ActorRef, _supervisor: ActorRef,
val path: ActorPath, val path: ActorPath,
@ -300,7 +299,7 @@ case class SerializedActorRef(hostname: String, port: Int, path: String) {
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
def readResolve(): AnyRef = { def readResolve(): AnyRef = {
if (app.value eq null) throw new IllegalStateException( if (app.value eq null) throw new IllegalStateException(
"Trying to deserialize a serialized ActorRef without an AkkaApplication in scope." + "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }") " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }")
app.value.provider.deserialize(this) match { app.value.provider.deserialize(this) match {
case Some(actor) actor case Some(actor) actor
@ -372,7 +371,7 @@ object DeadLetterActorRef {
val serialized = new SerializedDeadLetterActorRef val serialized = new SerializedDeadLetterActorRef
} }
class DeadLetterActorRef(val app: AkkaApplication) extends MinimalActorRef { class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
override val name: String = "dead-letter" override val name: String = "dead-letter"
@ -396,7 +395,7 @@ class DeadLetterActorRef(val app: AkkaApplication) extends MinimalActorRef {
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
} }
abstract class AskActorRef(protected val app: AkkaApplication)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef { abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef {
final val result = new DefaultPromise[Any](timeout)(dispatcher) final val result = new DefaultPromise[Any](timeout)(dispatcher)
// FIXME (actor path): put this under the tmp guardian supervisor // FIXME (actor path): put this under the tmp guardian supervisor
@ -432,4 +431,4 @@ abstract class AskActorRef(protected val app: AkkaApplication)(timeout: Timeout
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = app.provider.serialize(this) private def writeReplace(): AnyRef = app.provider.serialize(this)
} }

View file

@ -7,7 +7,6 @@ package akka.actor
import akka.config.ConfigurationException import akka.config.ConfigurationException
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.routing._ import akka.routing._
import akka.AkkaApplication
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.AkkaException import akka.AkkaException
import akka.dispatch._ import akka.dispatch._
@ -50,12 +49,12 @@ trait ActorRefProvider {
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef
private[akka] def terminationFuture: Future[AkkaApplication.ExitStatus] private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
} }
/** /**
* Interface implemented by AkkaApplication and AkkaContext, the only two places from which you can get fresh actors * Interface implemented by ActorSystem and AkkaContext, the only two places from which you can get fresh actors
*/ */
trait ActorRefFactory { trait ActorRefFactory {
@ -102,13 +101,13 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
/** /**
* Local ActorRef provider. * Local ActorRef provider.
*/ */
class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
val log = Logging(app.mainbus, this) val log = Logging(app.mainbus, this)
private[akka] val deployer: Deployer = new Deployer(app) private[akka] val deployer: Deployer = new Deployer(app)
val terminationFuture = new DefaultPromise[AkkaApplication.ExitStatus](Timeout.never)(app.dispatcher) val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher)
private[akka] val scheduler: Scheduler = { //TODO FIXME Make this configurable private[akka] val scheduler: Scheduler = { //TODO FIXME Make this configurable
val s = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) val s = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
@ -140,7 +139,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
protected[akka] override def postMessageToMailbox(msg: Any, sender: ActorRef) { protected[akka] override def postMessageToMailbox(msg: Any, sender: ActorRef) {
msg match { msg match {
case Failed(child, ex) child.stop() case Failed(child, ex) child.stop()
case ChildTerminated(child) terminationFuture.completeWithResult(AkkaApplication.Stopped) case ChildTerminated(child) terminationFuture.completeWithResult(ActorSystem.Stopped)
case _ log.error(this + " received unexpected message " + msg) case _ log.error(this + " received unexpected message " + msg)
} }
} }

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka package akka.actor
import akka.config._ import akka.config._
import akka.actor._ import akka.actor._
@ -13,11 +13,11 @@ import akka.dispatch.{ Dispatchers, Future }
import akka.util.Duration import akka.util.Duration
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.serialization.Serialization import akka.serialization.Serialization
import remote.{ RemoteAddress } import akka.remote.RemoteAddress
object AkkaApplication { object ActorSystem {
type AkkaConfig = a.AkkaConfig.type forSome { val a: AkkaApplication } type AkkaConfig = a.AkkaConfig.type forSome { val a: ActorSystem }
val Version = "2.0-SNAPSHOT" val Version = "2.0-SNAPSHOT"
@ -61,11 +61,11 @@ object AkkaApplication {
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
def apply(name: String, config: Configuration) = new AkkaApplication(name, config) def apply(name: String, config: Configuration) = new ActorSystem(name, config)
def apply(name: String): AkkaApplication = new AkkaApplication(name) def apply(name: String): ActorSystem = new ActorSystem(name)
def apply(): AkkaApplication = new AkkaApplication() def apply(): ActorSystem = new ActorSystem()
sealed trait ExitStatus sealed trait ExitStatus
case object Stopped extends ExitStatus case object Stopped extends ExitStatus
@ -73,12 +73,12 @@ object AkkaApplication {
} }
class AkkaApplication(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory { class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory {
def this(name: String) = this(name, AkkaApplication.defaultConfig) def this(name: String) = this(name, ActorSystem.defaultConfig)
def this() = this("default") def this() = this("default")
import AkkaApplication._ import ActorSystem._
object AkkaConfig { object AkkaConfig {
import config._ import config._

View file

@ -8,14 +8,13 @@ import java.io.File
import java.net.{ URL, URLClassLoader } import java.net.{ URL, URLClassLoader }
import java.util.jar.JarFile import java.util.jar.JarFile
import akka.util.Bootable import akka.util.Bootable
import akka.AkkaApplication
/** /**
* Handles all modules in the deploy directory (load and unload) * Handles all modules in the deploy directory (load and unload)
*/ */
trait BootableActorLoaderService extends Bootable { trait BootableActorLoaderService extends Bootable {
def app: AkkaApplication def app: ActorSystem
val BOOT_CLASSES = app.AkkaConfig.BootClasses val BOOT_CLASSES = app.AkkaConfig.BootClasses
lazy val applicationLoader = createApplicationClassLoader() lazy val applicationLoader = createApplicationClassLoader()
@ -67,4 +66,4 @@ trait BootableActorLoaderService extends Bootable {
/** /**
* Java API for the default JAX-RS/Mist Initializer * Java API for the default JAX-RS/Mist Initializer
*/ */
class DefaultBootableActorLoaderService(val app: AkkaApplication) extends BootableActorLoaderService class DefaultBootableActorLoaderService(val app: ActorSystem) extends BootableActorLoaderService

View file

@ -10,7 +10,7 @@ import java.util.concurrent.ConcurrentHashMap
import akka.event.Logging import akka.event.Logging
import akka.actor.DeploymentConfig._ import akka.actor.DeploymentConfig._
import akka.{ AkkaException, AkkaApplication } import akka.AkkaException
import akka.config.{ Configuration, ConfigurationException } import akka.config.{ Configuration, ConfigurationException }
import akka.util.Duration import akka.util.Duration
import java.net.InetSocketAddress import java.net.InetSocketAddress
@ -33,7 +33,7 @@ trait ActorDeployer {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Deployer(val app: AkkaApplication) extends ActorDeployer { class Deployer(val app: ActorSystem) extends ActorDeployer {
val deploymentConfig = new DeploymentConfig(app) val deploymentConfig = new DeploymentConfig(app)
val log = Logging(app.mainbus, this) val log = Logging(app.mainbus, this)

View file

@ -4,7 +4,6 @@
package akka.actor package akka.actor
import akka.AkkaApplication
import akka.util.Duration import akka.util.Duration
import akka.routing.{ RouterType, FailureDetectorType } import akka.routing.{ RouterType, FailureDetectorType }
import akka.routing.FailureDetectorType._ import akka.routing.FailureDetectorType._
@ -219,7 +218,7 @@ object DeploymentConfig {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class DeploymentConfig(val app: AkkaApplication) { class DeploymentConfig(val app: ActorSystem) {
import DeploymentConfig._ import DeploymentConfig._

View file

@ -6,7 +6,6 @@ package akka.actor
import akka.util._ import akka.util._
import scala.collection.mutable import scala.collection.mutable
import akka.AkkaApplication
import akka.event.Logging import akka.event.Logging
object FSM { object FSM {
@ -29,7 +28,7 @@ object FSM {
case object StateTimeout case object StateTimeout
case class TimeoutMarker(generation: Long) case class TimeoutMarker(generation: Long)
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: AkkaApplication) { case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: ActorSystem) {
private var ref: Option[Cancellable] = _ private var ref: Option[Cancellable] = _
def schedule(actor: ActorRef, timeout: Duration) { def schedule(actor: ActorRef, timeout: Duration) {

View file

@ -24,7 +24,6 @@ import scala.collection.immutable.Queue
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.util.continuations._ import scala.util.continuations._
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.AkkaApplication
object IO { object IO {
@ -291,7 +290,7 @@ private[akka] object IOWorker {
case object Shutdown extends Request case object Shutdown extends Request
} }
private[akka] class IOWorker(app: AkkaApplication, ioManager: ActorRef, val bufferSize: Int) { private[akka] class IOWorker(app: ActorSystem, ioManager: ActorRef, val bufferSize: Int) {
import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT }
import IOWorker._ import IOWorker._

View file

@ -10,14 +10,13 @@ import akka.util.{ Duration }
import java.util.concurrent.atomic.{ AtomicReference AtomVar } import java.util.concurrent.atomic.{ AtomicReference AtomVar }
import akka.serialization.{ Serializer, Serialization } import akka.serialization.{ Serializer, Serialization }
import akka.dispatch._ import akka.dispatch._
import akka.AkkaApplication
object TypedActor { object TypedActor {
/** /**
* This class represents a Method call, and has a reference to the Method to be called and the parameters to supply * This class represents a Method call, and has a reference to the Method to be called and the parameters to supply
* It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized * It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized
*/ */
case class MethodCall(app: AkkaApplication, method: Method, parameters: Array[AnyRef]) { case class MethodCall(app: ActorSystem, method: Method, parameters: Array[AnyRef]) {
def isOneWay = method.getReturnType == java.lang.Void.TYPE def isOneWay = method.getReturnType == java.lang.Void.TYPE
def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType) def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType)
@ -60,7 +59,7 @@ object TypedActor {
private def readResolve(): AnyRef = { private def readResolve(): AnyRef = {
val app = akka.serialization.Serialization.app.value val app = akka.serialization.Serialization.app.value
if (app eq null) throw new IllegalStateException( if (app eq null) throw new IllegalStateException(
"Trying to deserialize a SerializedMethodCall without an AkkaApplication in scope." + "Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }") " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }")
MethodCall(app, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { MethodCall(app, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match {
case null null case null null
@ -76,7 +75,7 @@ object TypedActor {
} }
private val selfReference = new ThreadLocal[AnyRef] private val selfReference = new ThreadLocal[AnyRef]
private val appReference = new ThreadLocal[AkkaApplication] private val appReference = new ThreadLocal[ActorSystem]
/** /**
* Returns the reference to the proxy when called inside a method call in a TypedActor * Returns the reference to the proxy when called inside a method call in a TypedActor
@ -265,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) * TypedActors needs, just like Actors, to be Stopped when they are no longer needed, use TypedActor.stop(proxy)
*/ */
class TypedActor(val app: AkkaApplication) { class TypedActor(val app: ActorSystem) {
import TypedActor.MethodCall import TypedActor.MethodCall
/** /**

View file

@ -12,7 +12,7 @@ import akka.serialization.Serializer
import akka.cluster.metrics._ import akka.cluster.metrics._
import akka.util.Duration import akka.util.Duration
import akka.util.duration._ import akka.util.duration._
import akka.{ AkkaException, AkkaApplication } import akka.AkkaException
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) {
*/ */
object NodeAddress { object NodeAddress {
def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName) def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName)
def apply(app: AkkaApplication): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename) def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename)
def unapply(other: Any) = other match { def unapply(other: Any) = other match {
case address: NodeAddress Some((address.clusterName, address.nodeName)) case address: NodeAddress Some((address.clusterName, address.nodeName))

View file

@ -11,7 +11,7 @@ import akka.config.Configuration
import akka.util.{ Duration, Switch, ReentrantGuard } import akka.util.{ Duration, Switch, ReentrantGuard }
import java.util.concurrent.ThreadPoolExecutor.{ AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy } import java.util.concurrent.ThreadPoolExecutor.{ AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy }
import akka.actor._ import akka.actor._
import akka.AkkaApplication import akka.actor.ActorSystem
import scala.annotation.tailrec import scala.annotation.tailrec
/** /**
@ -61,7 +61,7 @@ case class Supervise(child: ActorRef) extends SystemMessage // sent to superviso
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsMonitoring case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsMonitoring
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsMonitoring case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsMonitoring
final case class TaskInvocation(app: AkkaApplication, function: () Unit, cleanup: () Unit) extends Runnable { final case class TaskInvocation(app: ActorSystem, function: () Unit, cleanup: () Unit) extends Runnable {
def run() { def run() {
try { try {
function() function()
@ -78,13 +78,13 @@ object MessageDispatcher {
val SCHEDULED = 1 val SCHEDULED = 1
val RESCHEDULED = 2 val RESCHEDULED = 2
implicit def defaultDispatcher(implicit app: AkkaApplication) = app.dispatcher implicit def defaultDispatcher(implicit app: ActorSystem) = app.dispatcher
} }
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable { abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
import MessageDispatcher._ import MessageDispatcher._
protected val _tasks = new AtomicLong(0L) protected val _tasks = new AtomicLong(0L)
@ -338,7 +338,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable
/** /**
* Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig * Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig
*/ */
abstract class MessageDispatcherConfigurator(val app: AkkaApplication) { abstract class MessageDispatcherConfigurator(val app: ActorSystem) {
/** /**
* Returns an instance of MessageDispatcher given a Configuration * Returns an instance of MessageDispatcher given a Configuration
*/ */

View file

@ -9,7 +9,7 @@ import akka.actor.{ ActorCell, Actor, IllegalActorStateException }
import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
import java.util.{ Comparator, Queue } import java.util.{ Comparator, Queue }
import annotation.tailrec import annotation.tailrec
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -28,7 +28,7 @@ import akka.AkkaApplication
* @author Viktor Klang * @author Viktor Klang
*/ */
class BalancingDispatcher( class BalancingDispatcher(
_app: AkkaApplication, _app: ActorSystem,
_name: String, _name: String,
throughput: Int, throughput: Int,
throughputDeadlineTime: Int, throughputDeadlineTime: Int,

View file

@ -8,7 +8,7 @@ import akka.event.Logging.Warning
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue } import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue }
import akka.actor.{ ActorCell, ActorKilledException } import akka.actor.{ ActorCell, ActorKilledException }
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Default settings are: * Default settings are:
@ -64,7 +64,7 @@ import akka.AkkaApplication
* Larger values (or zero or negative) increase throughput, smaller values increase fairness * Larger values (or zero or negative) increase throughput, smaller values increase fairness
*/ */
class Dispatcher( class Dispatcher(
_app: AkkaApplication, _app: ActorSystem,
val name: String, val name: String,
val throughput: Int, val throughput: Int,
val throughputDeadlineTime: Int, val throughputDeadlineTime: Int,

View file

@ -9,7 +9,7 @@ import akka.actor.newUuid
import akka.util.{ Duration, ReflectiveAccess } import akka.util.{ Duration, ReflectiveAccess }
import akka.config.Configuration import akka.config.Configuration
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Scala API. Dispatcher factory. * Scala API. Dispatcher factory.
@ -43,7 +43,7 @@ import akka.AkkaApplication
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Dispatchers(val app: AkkaApplication) { class Dispatchers(val app: ActorSystem) {
val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType = val MailboxType: MailboxType =
if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox() if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
@ -206,7 +206,7 @@ class Dispatchers(val app: AkkaApplication) {
} }
} }
class DispatcherConfigurator(app: AkkaApplication) extends MessageDispatcherConfigurator(app) { class DispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) {
def configure(config: Configuration): MessageDispatcher = { def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new Dispatcher(app, configureThreadPool(config, threadPoolConfig new Dispatcher(app,
config.getString("name", newUuid.toString), config.getString("name", newUuid.toString),
@ -218,7 +218,7 @@ class DispatcherConfigurator(app: AkkaApplication) extends MessageDispatcherConf
} }
} }
class BalancingDispatcherConfigurator(app: AkkaApplication) extends MessageDispatcherConfigurator(app) { class BalancingDispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) {
def configure(config: Configuration): MessageDispatcher = { def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(app, configureThreadPool(config, threadPoolConfig new BalancingDispatcher(app,
config.getString("name", newUuid.toString), config.getString("name", newUuid.toString),

View file

@ -6,14 +6,14 @@ package akka.dispatch
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class PinnedDispatcher(_app: AkkaApplication, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long) class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long)
extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) { extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) {
@volatile @volatile
@ -34,6 +34,6 @@ class PinnedDispatcher(_app: AkkaApplication, _actor: ActorCell, _name: String,
} }
object PinnedDispatcher { object PinnedDispatcher {
def oneThread(app: AkkaApplication): ThreadPoolConfig = ThreadPoolConfig(app, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1) def oneThread(app: ActorSystem): ThreadPoolConfig = ThreadPoolConfig(app, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1)
} }

View file

@ -10,7 +10,7 @@ import atomic.{ AtomicLong, AtomicInteger }
import ThreadPoolExecutor.CallerRunsPolicy import ThreadPoolExecutor.CallerRunsPolicy
import akka.util.Duration import akka.util.Duration
import akka.event.Logging.{ Warning, Error } import akka.event.Logging.{ Warning, Error }
import akka.AkkaApplication import akka.actor.ActorSystem
object ThreadPoolConfig { object ThreadPoolConfig {
type Bounds = Int type Bounds = Int
@ -68,7 +68,7 @@ trait ExecutorServiceFactoryProvider {
/** /**
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher * A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
*/ */
case class ThreadPoolConfig(app: AkkaApplication, case class ThreadPoolConfig(app: ActorSystem,
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout, allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize, corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
@ -210,7 +210,7 @@ class MonitorableThread(runnable: Runnable, name: String)
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class BoundedExecutorDecorator(val app: AkkaApplication, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate { class BoundedExecutorDecorator(val app: ActorSystem, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate {
protected val semaphore = new Semaphore(bound) protected val semaphore = new Semaphore(bound)
override def execute(command: Runnable) = { override def execute(command: Runnable) = {

View file

@ -3,9 +3,9 @@
*/ */
package akka.event package akka.event
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props } import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem }
import akka.{ AkkaException, AkkaApplication } import akka.AkkaException
import akka.AkkaApplication.AkkaConfig import akka.actor.ActorSystem.AkkaConfig
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.config.ConfigurationException import akka.config.ConfigurationException
import akka.util.ReentrantGuard import akka.util.ReentrantGuard
@ -73,7 +73,7 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "StandardOutLogger started")) publish(Info(this, "StandardOutLogger started"))
} }
private[akka] def startDefaultLoggers(app: AkkaApplication, config: AkkaConfig) { private[akka] def startDefaultLoggers(app: ActorSystem, config: AkkaConfig) {
val level = levelFor(config.LogLevel) getOrElse { val level = levelFor(config.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel)) StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel))
ErrorLevel ErrorLevel
@ -128,7 +128,7 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "all default loggers stopped")) publish(Info(this, "all default loggers stopped"))
} }
private def addLogger(app: AkkaApplication, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { private def addLogger(app: ActorSystem, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
val actor = app.systemActorOf(Props(clazz), Props.randomName) val actor = app.systemActorOf(Props(clazz), Props.randomName)
actor ! InitializeLogger(this) actor ! InitializeLogger(this)
AllLogLevels filter (level >= _) foreach (l subscribe(actor, classFor(l))) AllLogLevels filter (level >= _) foreach (l subscribe(actor, classFor(l)))
@ -220,12 +220,12 @@ object Logging {
* Obtain LoggingAdapter for the given application and source object. The * Obtain LoggingAdapter for the given application and source object. The
* source object is used to identify the source of this logging channel. * source object is used to identify the source of this logging channel.
*/ */
def apply(app: AkkaApplication, source: AnyRef): LoggingAdapter = new BusLogging(app.mainbus, source) def apply(app: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(app.mainbus, source)
/** /**
* Java API: Obtain LoggingAdapter for the given application and source object. The * Java API: Obtain LoggingAdapter for the given application and source object. The
* source object is used to identify the source of this logging channel. * source object is used to identify the source of this logging channel.
*/ */
def getLogger(app: AkkaApplication, source: AnyRef): LoggingAdapter = apply(app, source) def getLogger(app: ActorSystem, source: AnyRef): LoggingAdapter = apply(app, source)
/** /**
* Obtain LoggingAdapter for the given event bus and source object. The * Obtain LoggingAdapter for the given event bus and source object. The
* source object is used to identify the source of this logging channel. * source object is used to identify the source of this logging channel.

View file

@ -4,7 +4,7 @@
package akka.event package akka.event
import akka.actor.{ ActorRef, Actor, Props } import akka.actor.{ ActorRef, Actor, Props }
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.actor.Terminated import akka.actor.Terminated
import akka.util.Subclassification import akka.util.Subclassification
@ -41,7 +41,7 @@ class MainBus(debug: Boolean = false) extends LoggingBus with SubchannelClassifi
super.unsubscribe(subscriber) super.unsubscribe(subscriber)
} }
def start(app: AkkaApplication) { def start(app: ActorSystem) {
reaper = app.systemActorOf(Props(new Actor { reaper = app.systemActorOf(Props(new Actor {
def receive = { def receive = {
case ref: ActorRef watch(ref) case ref: ActorRef watch(ref)

View file

@ -5,7 +5,7 @@
package akka.remote package akka.remote
import akka.actor._ import akka.actor._
import akka.{ AkkaException, AkkaApplication } import akka.AkkaException
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import java.io.{ PrintWriter, PrintStream } import java.io.{ PrintWriter, PrintStream }
@ -128,7 +128,7 @@ case class CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorExcept
override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter) override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter)
} }
abstract class RemoteSupport(val app: AkkaApplication) { abstract class RemoteSupport(val app: ActorSystem) {
/** /**
* Shuts down the remoting * Shuts down the remoting
*/ */

View file

@ -8,7 +8,7 @@ import akka.AkkaException
import akka.actor._ import akka.actor._
import akka.config.ConfigurationException import akka.config.ConfigurationException
import akka.dispatch.{ Future, MessageDispatcher } import akka.dispatch.{ Future, MessageDispatcher }
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.lang.reflect.InvocationTargetException import java.lang.reflect.InvocationTargetException
@ -92,7 +92,7 @@ object Routing {
/** /**
* An Abstract convenience implementation for building an ActorReference that uses a Router. * An Abstract convenience implementation for building an ActorReference that uses a Router.
*/ */
abstract private[akka] class AbstractRoutedActorRef(val app: AkkaApplication, val props: RoutedProps) extends UnsupportedActorRef { abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val props: RoutedProps) extends UnsupportedActorRef {
val router = props.routerFactory() val router = props.routerFactory()
override def postMessageToMailbox(message: Any, sender: ActorRef) = router.route(message)(sender) override def postMessageToMailbox(message: Any, sender: ActorRef) = router.route(message)(sender)
@ -104,7 +104,7 @@ abstract private[akka] class AbstractRoutedActorRef(val app: AkkaApplication, va
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors. * on (or more) of these actors.
*/ */
private[akka] class RoutedActorRef(app: AkkaApplication, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(app, routedProps) { private[akka] class RoutedActorRef(app: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(app, routedProps) {
val path = supervisor.path / name val path = supervisor.path / name

View file

@ -6,7 +6,7 @@ package akka.serialization
import akka.AkkaException import akka.AkkaException
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.AkkaApplication import akka.actor.ActorSystem
import scala.util.DynamicVariable import scala.util.DynamicVariable
import akka.remote.RemoteSupport import akka.remote.RemoteSupport
@ -16,7 +16,7 @@ case class NoSerializerFoundException(m: String) extends AkkaException(m)
* Serialization module. Contains methods for serialization and deserialization as well as * Serialization module. Contains methods for serialization and deserialization as well as
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file. * locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
*/ */
class Serialization(val app: AkkaApplication) { class Serialization(val app: ActorSystem) {
//TODO document me //TODO document me
def serialize(o: AnyRef): Either[Exception, Array[Byte]] = def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
@ -102,6 +102,6 @@ class Serialization(val app: AkkaApplication) {
object Serialization { object Serialization {
// TODO ensure that these are always set (i.e. withValue()) when doing deserialization // TODO ensure that these are always set (i.e. withValue()) when doing deserialization
val app = new DynamicVariable[AkkaApplication](null) val app = new DynamicVariable[ActorSystem](null)
} }

View file

@ -3,12 +3,12 @@
*/ */
package akka.util package akka.util
import akka.AkkaApplication import akka.actor.ActorSystem
/* /*
* This class is responsible for booting up a stack of bundles and then shutting them down * This class is responsible for booting up a stack of bundles and then shutting them down
*/ */
class AkkaLoader(app: AkkaApplication) { class AkkaLoader(app: ActorSystem) {
private val hasBooted = new Switch(false) private val hasBooted = new Switch(false)
@volatile @volatile
@ -86,6 +86,6 @@ class AkkaLoader(app: AkkaApplication) {
============================================================================== ==============================================================================
Running version %s Running version %s
============================================================================== ==============================================================================
""".format(AkkaApplication.Version)) """.format(ActorSystem.Version))
} }
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.util package akka.util
import akka.AkkaApplication import akka.actor.ActorSystem
trait Bootable { trait Bootable {
def onLoad() {} def onLoad() {}

View file

@ -7,7 +7,7 @@ package akka.util
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import TimeUnit._ import TimeUnit._
import java.lang.{ Long JLong, Double JDouble } import java.lang.{ Long JLong, Double JDouble }
import akka.AkkaApplication import akka.actor.ActorSystem
class TimerException(message: String) extends RuntimeException(message) class TimerException(message: String) extends RuntimeException(message)
@ -264,7 +264,7 @@ abstract class Duration extends Serializable {
def /(other: Duration): Double def /(other: Duration): Double
def unary_- : Duration def unary_- : Duration
def finite_? : Boolean def finite_? : Boolean
def dilated(implicit app: AkkaApplication): Duration = this * app.AkkaConfig.TestTimeFactor def dilated(implicit app: ActorSystem): Duration = this * app.AkkaConfig.TestTimeFactor
def min(other: Duration): Duration = if (this < other) this else other def min(other: Duration): Duration = if (this < other) this else other
def max(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) def sleep(): Unit = Thread.sleep(toMillis)

View file

@ -7,7 +7,7 @@ package akka.util
import akka.event.Logging.Error import akka.event.Logging.Error
import java.lang.management.ManagementFactory import java.lang.management.ManagementFactory
import javax.management.{ ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException } import javax.management.{ ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException }
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -18,7 +18,7 @@ object JMX {
def nameFor(hostname: String, service: String, bean: String): ObjectName = def nameFor(hostname: String, service: String, bean: String): ObjectName =
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_"))) new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
def register(name: ObjectName, mbean: AnyRef)(implicit app: AkkaApplication): Option[ObjectInstance] = try { def register(name: ObjectName, mbean: AnyRef)(implicit app: ActorSystem): Option[ObjectInstance] = try {
Some(mbeanServer.registerMBean(mbean, name)) Some(mbeanServer.registerMBean(mbean, name))
} catch { } catch {
case e: InstanceAlreadyExistsException case e: InstanceAlreadyExistsException
@ -28,7 +28,7 @@ object JMX {
None None
} }
def unregister(mbean: ObjectName)(implicit app: AkkaApplication) = try { def unregister(mbean: ObjectName)(implicit app: ActorSystem) = try {
mbeanServer.unregisterMBean(mbean) mbeanServer.unregisterMBean(mbean)
} catch { } catch {
case e: InstanceNotFoundException {} case e: InstanceNotFoundException {}

View file

@ -11,7 +11,7 @@ import akka.config.ModuleNotAvailableException
import akka.event.Logging.Debug import akka.event.Logging.Debug
import akka.cluster.ClusterNode import akka.cluster.ClusterNode
import akka.routing.{ RoutedProps, Router } import akka.routing.{ RoutedProps, Router }
import akka.AkkaApplication import akka.actor.ActorSystem
object ReflectiveAccess { object ReflectiveAccess {
@ -117,7 +117,7 @@ object ReflectiveAccess {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class ReflectiveAccess(val app: AkkaApplication) { class ReflectiveAccess(val app: ActorSystem) {
import ReflectiveAccess._ import ReflectiveAccess._
@ -129,7 +129,7 @@ class ReflectiveAccess(val app: AkkaApplication) {
} }
def createProvider: ActorRefProvider = { def createProvider: ActorRefProvider = {
val params: Array[Class[_]] = Array(classOf[AkkaApplication]) val params: Array[Class[_]] = Array(classOf[ActorSystem])
val args: Array[AnyRef] = Array(app) val args: Array[AnyRef] = Array(app)
createInstance[ActorRefProvider](providerClass, params, args) match { createInstance[ActorRefProvider](providerClass, params, args) match {

View file

@ -5,8 +5,7 @@
//#imports //#imports
package akka.tutorial.first.scala package akka.tutorial.first.scala
import akka.AkkaApplication import akka.actor.{ Actor, ActorSystem, PoisonPill }
import akka.actor.{ Actor, PoisonPill }
import akka.routing.Routing.Broadcast import akka.routing.Routing.Broadcast
import akka.routing.{ RoutedProps, Routing } import akka.routing.{ RoutedProps, Routing }
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
@ -15,7 +14,7 @@ import java.util.concurrent.CountDownLatch
//#app //#app
object Pi extends App { object Pi extends App {
val app = AkkaApplication() val app = ActorSystem()
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)

View file

@ -11,7 +11,7 @@ import scala.annotation.tailrec
import System.{ currentTimeMillis newTimestamp } import System.{ currentTimeMillis newTimestamp }
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper: * Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper:
@ -25,7 +25,7 @@ import akka.AkkaApplication
*/ */
class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 1000) { class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 1000) {
def this(app: AkkaApplication) { def this(app: ActorSystem) {
this( this(
app.config.getInt("akka.remote.failure-detector.theshold", 8), app.config.getInt("akka.remote.failure-detector.theshold", 8),
app.config.getInt("akka.remote.failure-detector.max-sample-size", 1000)) app.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))

View file

@ -7,17 +7,17 @@ package akka.remote
import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol._
import akka.serialization.Serialization import akka.serialization.Serialization
import com.google.protobuf.ByteString import com.google.protobuf.ByteString
import akka.AkkaApplication import akka.actor.ActorSystem
object MessageSerializer { object MessageSerializer {
def deserialize(app: AkkaApplication, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = { def deserialize(app: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
val clazz = loadManifest(classLoader, messageProtocol) val clazz = loadManifest(classLoader, messageProtocol)
app.serialization.deserialize(messageProtocol.getMessage.toByteArray, app.serialization.deserialize(messageProtocol.getMessage.toByteArray,
clazz, classLoader).fold(x throw x, identity) clazz, classLoader).fold(x throw x, identity)
} }
def serialize(app: AkkaApplication, message: AnyRef): MessageProtocol = { def serialize(app: ActorSystem, message: AnyRef): MessageProtocol = {
val builder = MessageProtocol.newBuilder val builder = MessageProtocol.newBuilder
val bytes = app.serialization.serialize(message).fold(x throw x, identity) val bytes = app.serialization.serialize(message).fold(x throw x, identity)
builder.setMessage(ByteString.copyFrom(bytes)) builder.setMessage(ByteString.copyFrom(bytes))

View file

@ -7,7 +7,7 @@ package akka.remote
import scala.collection.mutable import scala.collection.mutable
import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid } import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid }
import akka.actor.Actor._ import akka.actor.Actor._
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc. * Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
@ -58,7 +58,7 @@ object NetworkEventStream {
} }
} }
class NetworkEventStream(val app: AkkaApplication) { class NetworkEventStream(val app: ActorSystem) {
import NetworkEventStream._ import NetworkEventStream._

View file

@ -4,7 +4,7 @@
package akka.remote package akka.remote
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.actor._ import akka.actor._
import akka.event.Logging import akka.event.Logging
import akka.actor.Status._ import akka.actor.Status._
@ -27,7 +27,7 @@ import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Remote(val app: AkkaApplication) { class Remote(val app: ActorSystem) {
val log = Logging(app, this) val log = Logging(app, this)
@ -266,7 +266,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
trait RemoteMarshallingOps { trait RemoteMarshallingOps {
def app: AkkaApplication def app: ActorSystem
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
val arp = AkkaRemoteProtocol.newBuilder val arp = AkkaRemoteProtocol.newBuilder

View file

@ -4,7 +4,7 @@
package akka.remote package akka.remote
import akka.{ AkkaException, AkkaApplication } import akka.AkkaException
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._
import akka.actor.Status._ import akka.actor.Status._
@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
val log = Logging(app, this) val log = Logging(app, this)

View file

@ -6,7 +6,7 @@ package akka.remote
import akka.actor._ import akka.actor._
import akka.routing._ import akka.routing._
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.event.Logging import akka.event.Logging
import scala.collection.immutable.Map import scala.collection.immutable.Map
@ -20,7 +20,7 @@ import java.util.concurrent.atomic.AtomicReference
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class RemoteConnectionManager( class RemoteConnectionManager(
app: AkkaApplication, app: ActorSystem,
remote: Remote, remote: Remote,
initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef]) initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef])
extends ConnectionManager { extends ConnectionManager {

View file

@ -21,7 +21,7 @@ import java.net.InetSocketAddress
import java.util.concurrent._ import java.util.concurrent._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import akka.AkkaException import akka.AkkaException
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.event.Logging import akka.event.Logging
import org.jboss.netty.channel._ import org.jboss.netty.channel._
@ -348,7 +348,7 @@ class ActiveRemoteClientHandler(
/** /**
* Provides the implementation of the Netty remote support * Provides the implementation of the Netty remote support
*/ */
class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with RemoteMarshallingOps { class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit) val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit) val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit)

View file

@ -3,9 +3,8 @@ package sample.fsm.dining.become
//Akka adaptation of //Akka adaptation of
//http://www.dalnefre.com/wp/2010/08/dining-philosophers-in-humus/ //http://www.dalnefre.com/wp/2010/08/dining-philosophers-in-humus/
import akka.actor.{ ActorRef, Actor } import akka.actor.{ ActorRef, Actor, ActorSystem }
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.AkkaApplication
/* /*
* First we define our messages, they basically speak for themselves * First we define our messages, they basically speak for themselves
@ -123,7 +122,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
* Alright, here's our test-harness * Alright, here's our test-harness
*/ */
object DiningHakkers { object DiningHakkers {
val app = AkkaApplication() val app = ActorSystem()
def run { def run {
//Create 5 chopsticks //Create 5 chopsticks
val chopsticks = for (i 1 to 5) yield app.actorOf(new Chopstick("Chopstick " + i)) val chopsticks = for (i 1 to 5) yield app.actorOf(new Chopstick("Chopstick " + i))

View file

@ -1,10 +1,9 @@
package sample.fsm.dining.fsm package sample.fsm.dining.fsm
import akka.actor.{ ActorRef, Actor, FSM } import akka.actor.{ ActorRef, Actor, FSM, ActorSystem }
import akka.actor.FSM._ import akka.actor.FSM._
import akka.util.Duration import akka.util.Duration
import akka.util.duration._ import akka.util.duration._
import akka.AkkaApplication
/* /*
* Some messages for the chopstick * Some messages for the chopstick
@ -164,7 +163,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit
*/ */
object DiningHakkersOnFsm { object DiningHakkersOnFsm {
val app = AkkaApplication() val app = ActorSystem()
def run = { def run = {
// Create 5 chopsticks // Create 5 chopsticks

View file

@ -4,7 +4,7 @@
package akka.agent package akka.agent
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.actor._ import akka.actor._
import akka.stm._ import akka.stm._
import akka.japi.{ Function JFunc, Procedure JProc } import akka.japi.{ Function JFunc, Procedure JProc }
@ -20,7 +20,7 @@ private[akka] case object Get
* Factory method for creating an Agent. * Factory method for creating an Agent.
*/ */
object Agent { object Agent {
def apply[T](initialValue: T)(implicit app: AkkaApplication) = new Agent(initialValue, app) def apply[T](initialValue: T)(implicit app: ActorSystem) = new Agent(initialValue, app)
} }
/** /**
@ -93,7 +93,7 @@ object Agent {
* agent4.close * agent4.close
* }}} * }}}
*/ */
class Agent[T](initialValue: T, app: AkkaApplication) { class Agent[T](initialValue: T, app: ActorSystem) {
private[akka] val ref = Ref(initialValue) private[akka] val ref = Ref(initialValue)
private[akka] val updater = app.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? private[akka] val updater = app.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow?

View file

@ -1,6 +1,5 @@
package akka.stm.example; package akka.stm.example;
import akka.AkkaApplication;
import akka.stm.*; import akka.stm.*;
import akka.actor.*; import akka.actor.*;
@ -10,7 +9,7 @@ public class EitherOrElseExample {
System.out.println("EitherOrElse example"); System.out.println("EitherOrElse example");
System.out.println(); System.out.println();
AkkaApplication application = new AkkaApplication("UntypedTransactorExample"); ActorSystem application = new ActorSystem("UntypedTransactorExample");
final Ref<Integer> left = new Ref<Integer>(100); final Ref<Integer> left = new Ref<Integer>(100);
final Ref<Integer> right = new Ref<Integer>(100); final Ref<Integer> right = new Ref<Integer>(100);

View file

@ -1,6 +1,6 @@
package akka.stm.example; package akka.stm.example;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import akka.stm.*; import akka.stm.*;
import akka.actor.*; import akka.actor.*;
@ -10,7 +10,7 @@ public class RetryExample {
System.out.println("Retry example"); System.out.println("Retry example");
System.out.println(); System.out.println();
AkkaApplication application = new AkkaApplication("RetryExample"); ActorSystem application = new ActorSystem("RetryExample");
final Ref<Double> account1 = new Ref<Double>(100.0); final Ref<Double> account1 = new Ref<Double>(100.0);
final Ref<Double> account2 = new Ref<Double>(100.0); final Ref<Double> account2 = new Ref<Double>(100.0);

View file

@ -1,6 +1,6 @@
package akka.transactor.example; package akka.transactor.example;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Props; import akka.actor.Props;
import akka.dispatch.Future; import akka.dispatch.Future;
@ -12,7 +12,7 @@ public class UntypedCoordinatedExample {
System.out.println("Untyped transactor example"); System.out.println("Untyped transactor example");
System.out.println(); System.out.println();
AkkaApplication application = new AkkaApplication("UntypedCoordinatedExample"); ActorSystem application = new ActorSystem("UntypedCoordinatedExample");
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));

View file

@ -1,6 +1,6 @@
package akka.transactor.example; package akka.transactor.example;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Props; import akka.actor.Props;
import akka.dispatch.Future; import akka.dispatch.Future;
@ -11,7 +11,7 @@ public class UntypedTransactorExample {
System.out.println("Untyped transactor example"); System.out.println("Untyped transactor example");
System.out.println(); System.out.println();
AkkaApplication application = new AkkaApplication("UntypedTransactorExample"); ActorSystem application = new ActorSystem("UntypedTransactorExample");
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class)); ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class));
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class)); ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class));

View file

@ -6,7 +6,7 @@ import org.junit.After;
import org.junit.Test; import org.junit.Test;
import org.junit.Before; import org.junit.Before;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import akka.transactor.Coordinated; import akka.transactor.Coordinated;
import akka.actor.Actors; import akka.actor.Actors;
import akka.actor.ActorRef; import akka.actor.ActorRef;
@ -30,7 +30,7 @@ import scala.collection.JavaConverters;
import scala.collection.Seq; import scala.collection.Seq;
public class UntypedCoordinatedIncrementTest { public class UntypedCoordinatedIncrementTest {
AkkaApplication application = new AkkaApplication("UntypedCoordinatedIncrementTest"); ActorSystem application = new ActorSystem("UntypedCoordinatedIncrementTest");
List<ActorRef> counters; List<ActorRef> counters;
ActorRef failer; ActorRef failer;

View file

@ -4,7 +4,7 @@ import static org.junit.Assert.*;
import org.junit.Test; import org.junit.Test;
import org.junit.Before; import org.junit.Before;
import akka.AkkaApplication; import akka.actor.ActorSystem;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Actors; import akka.actor.Actors;
import akka.actor.Props; import akka.actor.Props;
@ -27,7 +27,7 @@ import scala.collection.JavaConverters;
import scala.collection.Seq; import scala.collection.Seq;
public class UntypedTransactorTest { public class UntypedTransactorTest {
AkkaApplication application = new AkkaApplication("UntypedTransactorTest"); ActorSystem application = new ActorSystem("UntypedTransactorTest");
List<ActorRef> counters; List<ActorRef> counters;
ActorRef failer; ActorRef failer;

View file

@ -3,7 +3,7 @@ package akka.agent.test
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.actor.Timeout import akka.actor.Timeout
import akka.agent.Agent import akka.agent.Agent
import akka.stm._ import akka.stm._
@ -20,7 +20,7 @@ class CountDownFunction[A](num: Int = 1) extends Function1[A, A] {
class AgentSpec extends WordSpec with MustMatchers { class AgentSpec extends WordSpec with MustMatchers {
implicit val app = AkkaApplication("AgentSpec") implicit val app = ActorSystem("AgentSpec")
implicit val timeout = Timeout(5.seconds.dilated) implicit val timeout = Timeout(5.seconds.dilated)
"Agent" should { "Agent" should {

View file

@ -9,14 +9,14 @@ import org.scalatest.WordSpec
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.AkkaApplication import akka.actor.ActorSystem
@RunWith(classOf[JUnitRunner]) @RunWith(classOf[JUnitRunner])
class ConfigSpec extends WordSpec with MustMatchers { class ConfigSpec extends WordSpec with MustMatchers {
"The default configuration file (i.e. akka-reference.conf)" should { "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 { "contain all configuration properties for akka-stm that are used in code with their correct defaults" in {
val config = AkkaApplication("ConfigSpec").config val config = ActorSystem("ConfigSpec").config
import config._ import config._

View file

@ -2,7 +2,7 @@ package akka.transactor.test
import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterAll
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.transactor.Coordinated import akka.transactor.Coordinated
import akka.actor._ import akka.actor._
import akka.stm.{ Ref, TransactionFactory } import akka.stm.{ Ref, TransactionFactory }

View file

@ -4,7 +4,7 @@ import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterAll
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.transactor.Coordinated import akka.transactor.Coordinated
import akka.actor._ import akka.actor._
import akka.stm._ import akka.stm._

View file

@ -3,7 +3,7 @@ package akka.transactor.test
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.transactor.Transactor import akka.transactor.Transactor
import akka.actor._ import akka.actor._
import akka.stm._ import akka.stm._

View file

@ -12,7 +12,7 @@ import java.lang.ref.WeakReference
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.dispatch._ import akka.dispatch._
import akka.AkkaApplication import akka.actor.ActorSystem
/* /*
* Locking rules: * Locking rules:
@ -104,7 +104,7 @@ private[testkit] object CallingThreadDispatcher {
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.1 * @since 1.1
*/ */
class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling-thread") extends MessageDispatcher(_app) { class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thread") extends MessageDispatcher(_app) {
import CallingThreadDispatcher._ import CallingThreadDispatcher._
protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this, actor) protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this, actor)

View file

@ -8,7 +8,7 @@ import akka.actor._
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.actor.Props._ import akka.actor.Props._
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it
@ -18,7 +18,7 @@ import akka.AkkaApplication
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.1 * @since 1.1
*/ */
class TestActorRef[T <: Actor](_app: AkkaApplication, _props: Props, _supervisor: ActorRef, name: String) class TestActorRef[T <: Actor](_app: ActorSystem, _props: Props, _supervisor: ActorRef, name: String)
extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_app)), _supervisor, _supervisor.path / name, false) { extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_app)), _supervisor, _supervisor.path / name, false) {
/** /**
* Directly inject messages into actor receive behavior. Any exceptions * Directly inject messages into actor receive behavior. Any exceptions
@ -41,15 +41,15 @@ class TestActorRef[T <: Actor](_app: AkkaApplication, _props: Props, _supervisor
object TestActorRef { object TestActorRef {
def apply[T <: Actor](factory: T)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](Props(factory), Props.randomName) def apply[T <: Actor](factory: T)(implicit app: ActorSystem): TestActorRef[T] = apply[T](Props(factory), Props.randomName)
def apply[T <: Actor](factory: T, name: String)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](Props(factory), name) def apply[T <: Actor](factory: T, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](Props(factory), name)
def apply[T <: Actor](props: Props)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](props, Props.randomName) def apply[T <: Actor](props: Props)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, Props.randomName)
def apply[T <: Actor](props: Props, name: String)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](props, app.guardian, name) def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, app.guardian, name)
def apply[T <: Actor](props: Props, supervisor: ActorRef, givenName: String)(implicit app: AkkaApplication): TestActorRef[T] = { def apply[T <: Actor](props: Props, supervisor: ActorRef, givenName: String)(implicit app: ActorSystem): TestActorRef[T] = {
val name: String = givenName match { val name: String = givenName match {
case null | Props.randomName newUuid.toString case null | Props.randomName newUuid.toString
case given given case given given
@ -57,9 +57,9 @@ object TestActorRef {
new TestActorRef(app, props, supervisor, name) new TestActorRef(app, props, supervisor, name)
} }
def apply[T <: Actor](implicit m: Manifest[T], app: AkkaApplication): TestActorRef[T] = apply[T](Props.randomName) def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](Props.randomName)
def apply[T <: Actor](name: String)(implicit m: Manifest[T], app: AkkaApplication): TestActorRef[T] = apply[T](Props({ def apply[T <: Actor](name: String)(implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](Props({
import ReflectiveAccess.{ createInstance, noParams, noArgs } import ReflectiveAccess.{ createInstance, noParams, noArgs }
createInstance[T](m.erasure, noParams, noArgs) match { createInstance[T](m.erasure, noParams, noArgs) match {
case Right(value) value case Right(value) value

View file

@ -6,7 +6,7 @@ package akka.testkit
import akka.util.Duration import akka.util.Duration
import java.util.concurrent.{ CyclicBarrier, TimeUnit, TimeoutException } import java.util.concurrent.{ CyclicBarrier, TimeUnit, TimeoutException }
import akka.AkkaApplication import akka.actor.ActorSystem
class TestBarrierTimeoutException(message: String) extends RuntimeException(message) class TestBarrierTimeoutException(message: String) extends RuntimeException(message)
@ -25,9 +25,9 @@ object TestBarrier {
class TestBarrier(count: Int) { class TestBarrier(count: Int) {
private val barrier = new CyclicBarrier(count) private val barrier = new CyclicBarrier(count)
def await()(implicit app: AkkaApplication): Unit = await(TestBarrier.DefaultTimeout) def await()(implicit app: ActorSystem): Unit = await(TestBarrier.DefaultTimeout)
def await(timeout: Duration)(implicit app: AkkaApplication) { def await(timeout: Duration)(implicit app: ActorSystem) {
try { try {
barrier.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) barrier.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS)
} catch { } catch {

View file

@ -9,7 +9,7 @@ import akka.actor.Actor
import akka.event.Logging._ import akka.event.Logging._
import akka.event.Logging import akka.event.Logging
import akka.util.Duration import akka.util.Duration
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Implementation helpers of the EventFilter facilities: send `Mute` * Implementation helpers of the EventFilter facilities: send `Mute`
@ -78,7 +78,7 @@ abstract class EventFilter(occurrences: Int) {
* Apply this filter while executing the given code block. Care is taken to * Apply this filter while executing the given code block. Care is taken to
* remove the filter when the block is finished or aborted. * remove the filter when the block is finished or aborted.
*/ */
def intercept[T](code: T)(implicit app: AkkaApplication): T = { def intercept[T](code: T)(implicit app: ActorSystem): T = {
app.mainbus publish TestEvent.Mute(this) app.mainbus publish TestEvent.Mute(this)
try { try {
val result = code val result = code

View file

@ -7,7 +7,7 @@ package akka.testkit
import akka.actor._ import akka.actor._
import akka.util._ import akka.util._
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* This is a specialised form of the TestActorRef with support for querying and * This is a specialised form of the TestActorRef with support for querying and
@ -34,7 +34,7 @@ import akka.AkkaApplication
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.2 * @since 1.2
*/ */
class TestFSMRef[S, D, T <: Actor](app: AkkaApplication, props: Props, supervisor: ActorRef, name: String)(implicit ev: T <:< FSM[S, D]) class TestFSMRef[S, D, T <: Actor](app: ActorSystem, props: Props, supervisor: ActorRef, name: String)(implicit ev: T <:< FSM[S, D])
extends TestActorRef(app, props, supervisor, name) { extends TestActorRef(app, props, supervisor, name) {
private def fsm: T = underlyingActor private def fsm: T = underlyingActor
@ -80,9 +80,9 @@ class TestFSMRef[S, D, T <: Actor](app: AkkaApplication, props: Props, superviso
object TestFSMRef { object TestFSMRef {
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: AkkaApplication): TestFSMRef[S, D, T] = def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
new TestFSMRef(app, Props(creator = () factory), app.guardian, Props.randomName) new TestFSMRef(app, Props(creator = () factory), app.guardian, Props.randomName)
def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: AkkaApplication): TestFSMRef[S, D, T] = def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
new TestFSMRef(app, Props(creator = () factory), app.guardian, name) new TestFSMRef(app, Props(creator = () factory), app.guardian, name)
} }

View file

@ -10,7 +10,7 @@ import akka.util.duration._
import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic } import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic }
import atomic.AtomicInteger import atomic.AtomicInteger
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.AkkaApplication import akka.actor.ActorSystem
object TestActor { object TestActor {
type Ignore = Option[PartialFunction[AnyRef, Boolean]] type Ignore = Option[PartialFunction[AnyRef, Boolean]]
@ -86,7 +86,7 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor with FSM[
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.1 * @since 1.1
*/ */
class TestKit(_app: AkkaApplication) { class TestKit(_app: ActorSystem) {
import TestActor.{ Message, RealMessage, NullMessage } import TestActor.{ Message, RealMessage, NullMessage }
@ -589,7 +589,7 @@ object TestKit {
/** /**
* TestKit-based probe which allows sending, reception and reply. * TestKit-based probe which allows sending, reception and reply.
*/ */
class TestProbe(_application: AkkaApplication) extends TestKit(_application) { class TestProbe(_application: ActorSystem) extends TestKit(_application) {
/** /**
* Shorthand to get the testActor. * Shorthand to get the testActor.
@ -620,7 +620,7 @@ class TestProbe(_application: AkkaApplication) extends TestKit(_application) {
} }
object TestProbe { object TestProbe {
def apply()(implicit app: AkkaApplication) = new TestProbe(app) def apply()(implicit app: ActorSystem) = new TestProbe(app)
} }
trait ImplicitSender { this: TestKit trait ImplicitSender { this: TestKit

View file

@ -6,7 +6,7 @@ package akka.testkit
import akka.util.Duration import akka.util.Duration
import java.util.concurrent.{ CountDownLatch, TimeUnit } import java.util.concurrent.{ CountDownLatch, TimeUnit }
import akka.AkkaApplication import akka.actor.ActorSystem
class TestLatchTimeoutException(message: String) extends RuntimeException(message) class TestLatchTimeoutException(message: String) extends RuntimeException(message)
class TestLatchNoTimeoutException(message: String) extends RuntimeException(message) class TestLatchNoTimeoutException(message: String) extends RuntimeException(message)
@ -21,10 +21,10 @@ class TestLatchNoTimeoutException(message: String) extends RuntimeException(mess
object TestLatch { object TestLatch {
val DefaultTimeout = Duration(5, TimeUnit.SECONDS) val DefaultTimeout = Duration(5, TimeUnit.SECONDS)
def apply(count: Int = 1)(implicit app: AkkaApplication) = new TestLatch(count) def apply(count: Int = 1)(implicit app: ActorSystem) = new TestLatch(count)
} }
class TestLatch(count: Int = 1)(implicit app: AkkaApplication) { class TestLatch(count: Int = 1)(implicit app: ActorSystem) {
private var latch = new CountDownLatch(count) private var latch = new CountDownLatch(count)
def countDown() = latch.countDown() def countDown() = latch.countDown()

View file

@ -1,10 +1,11 @@
package akka package akka
import akka.actor.ActorSystem
import akka.util.Duration import akka.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
package object testkit { package object testkit {
def filterEvents[T](eventFilters: Iterable[EventFilter])(block: T)(implicit app: AkkaApplication): T = { def filterEvents[T](eventFilters: Iterable[EventFilter])(block: T)(implicit app: ActorSystem): T = {
def now = System.currentTimeMillis def now = System.currentTimeMillis
app.mainbus.publish(TestEvent.Mute(eventFilters.toSeq)) app.mainbus.publish(TestEvent.Mute(eventFilters.toSeq))
@ -22,7 +23,7 @@ package object testkit {
} }
} }
def filterEvents[T](eventFilters: EventFilter*)(block: T)(implicit app: AkkaApplication): T = filterEvents(eventFilters.toSeq)(block) def filterEvents[T](eventFilters: EventFilter*)(block: T)(implicit app: ActorSystem): T = filterEvents(eventFilters.toSeq)(block)
def filterException[T <: Throwable](block: Unit)(implicit app: AkkaApplication, m: Manifest[T]): Unit = EventFilter[T]() intercept (block) def filterException[T <: Throwable](block: Unit)(implicit app: ActorSystem, m: Manifest[T]): Unit = EventFilter[T]() intercept (block)
} }

View file

@ -6,14 +6,14 @@ package akka.testkit
import akka.config.Configuration import akka.config.Configuration
import org.scalatest.{ WordSpec, BeforeAndAfterAll } import org.scalatest.{ WordSpec, BeforeAndAfterAll }
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.AkkaApplication import akka.actor.ActorSystem
import akka.actor.{ Actor, ActorRef, Props } import akka.actor.{ Actor, ActorRef, Props }
import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher
import akka.event.{ Logging, LoggingAdapter } import akka.event.{ Logging, LoggingAdapter }
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.FutureTimeoutException import akka.dispatch.FutureTimeoutException
abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll {
val log: LoggingAdapter = Logging(app.mainbus, this) val log: LoggingAdapter = Logging(app.mainbus, this)
@ -34,7 +34,7 @@ abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication())
protected def atTermination() {} protected def atTermination() {}
def this(config: Configuration) = this(new AkkaApplication(getClass.getSimpleName, AkkaApplication.defaultConfig ++ config)) def this(config: Configuration) = this(new ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config))
def actorOf(props: Props): ActorRef = app.actorOf(props) def actorOf(props: Props): ActorRef = app.actorOf(props)
@ -53,8 +53,8 @@ abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication())
class AkkaSpecSpec extends WordSpec with MustMatchers { class AkkaSpecSpec extends WordSpec with MustMatchers {
"An AkkaSpec" must { "An AkkaSpec" must {
"terminate all actors" in { "terminate all actors" in {
import AkkaApplication.defaultConfig import ActorSystem.defaultConfig
val app = AkkaApplication("test", defaultConfig ++ Configuration( val app = ActorSystem("test", defaultConfig ++ Configuration(
"akka.actor.debug.lifecycle" -> true, "akka.loglevel" -> "DEBUG")) "akka.actor.debug.lifecycle" -> true, "akka.loglevel" -> "DEBUG"))
val spec = new AkkaSpec(app) { val spec = new AkkaSpec(app) {
val ref = Seq(testActor, app.actorOf(Props.empty, "name")) val ref = Seq(testActor, app.actorOf(Props.empty, "name"))

View file

@ -9,7 +9,7 @@ import akka.actor._
import akka.event.Logging.Warning import akka.event.Logging.Warning
import akka.dispatch.{ Future, Promise } import akka.dispatch.{ Future, Promise }
import akka.util.duration._ import akka.util.duration._
import akka.AkkaApplication import akka.actor.ActorSystem
/** /**
* Test whether TestActorRef behaves as an ActorRef should, besides its own spec. * Test whether TestActorRef behaves as an ActorRef should, besides its own spec.

View file

@ -9,6 +9,7 @@ import static java.util.Arrays.asList;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Actors; import akka.actor.Actors;
import akka.actor.ActorSystem;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import akka.routing.RoutedProps; import akka.routing.RoutedProps;
@ -21,11 +22,9 @@ import scala.collection.JavaConversions;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import akka.AkkaApplication;
public class Pi { public class Pi {
private static final AkkaApplication app = new AkkaApplication(); private static final ActorSystem app = new ActorSystem();
public static void main(String[] args) throws Exception { public static void main(String[] args) throws Exception {
Pi pi = new Pi(); Pi pi = new Pi();

View file

@ -4,16 +4,15 @@
package akka.tutorial.first.scala package akka.tutorial.first.scala
import akka.actor.{ Actor, PoisonPill } import akka.actor.{ Actor, PoisonPill, ActorSystem }
import Actor._ import Actor._
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import akka.routing.Routing.Broadcast import akka.routing.Routing.Broadcast
import akka.routing.{ RoutedProps, Routing } import akka.routing.{ RoutedProps, Routing }
import akka.AkkaApplication
object Pi extends App { object Pi extends App {
val app = AkkaApplication() val app = ActorSystem()
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)

View file

@ -8,13 +8,13 @@ import static akka.actor.Actors.poisonPill;
import static java.lang.System.currentTimeMillis; import static java.lang.System.currentTimeMillis;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import akka.AkkaApplication;
import akka.routing.RoutedProps; import akka.routing.RoutedProps;
import akka.routing.Routing; import akka.routing.Routing;
import akka.routing.LocalConnectionManager; import akka.routing.LocalConnectionManager;
import scala.Option; import scala.Option;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Actors; import akka.actor.Actors;
import akka.actor.ActorSystem;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import akka.dispatch.Future; import akka.dispatch.Future;
@ -26,7 +26,7 @@ import java.util.LinkedList;
public class Pi { public class Pi {
private static final AkkaApplication app = new AkkaApplication(); private static final ActorSystem app = new ActorSystem();
public static void main(String[] args) throws Exception { public static void main(String[] args) throws Exception {
Pi pi = new Pi(); Pi pi = new Pi();

View file

@ -9,12 +9,11 @@ import akka.event.Logging
import System.{ currentTimeMillis now } import System.{ currentTimeMillis now }
import akka.routing.Routing.Broadcast import akka.routing.Routing.Broadcast
import akka.routing._ import akka.routing._
import akka.AkkaApplication import akka.actor.{ ActorRef, Timeout, Actor, PoisonPill, ActorSystem }
import akka.actor.{ ActorRef, Timeout, Actor, PoisonPill }
object Pi extends App { object Pi extends App {
val app = AkkaApplication() val app = ActorSystem()
val log = Logging(app, this) val log = Logging(app, this)
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)