clean up initialization of ActorSystem, fixes #1050

- create ActorSystemImpl trait to make ActorSystem fully abstract
- add Java API for constructing (ActorSystem.create(...))
- only go through factory methods because .start() has become necessary
- rename all user-facing occurrences of “app” to “system” (Actor trait
  and TestKit/AkkaSpec)
- pass ActorSystemImpl to ActorRefs upon creation, which means that
  actorOf() and friends need such an argument, which must be provided to
  the ActorRefProvider by the ActorRefFactory implementation
This commit is contained in:
Roland 2011-11-16 17:18:36 +01:00
parent 6d85572ecc
commit 648661c548
83 changed files with 494 additions and 390 deletions

View file

@ -9,7 +9,7 @@ import static org.junit.Assert.*;
public class JavaAPI {
private ActorSystem app = new ActorSystem();
private ActorSystem app = ActorSystem.create();
@Test void mustBeAbleToCreateActorRefFromClass() {
ActorRef ref = app.actorOf(JavaAPITestActor.class);

View file

@ -19,7 +19,7 @@ import scala.Right;
public class JavaFutureTests {
private final ActorSystem app = new ActorSystem();
private final ActorSystem app = ActorSystem.create();
private final Timeout t = app.AkkaConfig().ActorTimeout();
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);

View file

@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec {
out.flush
out.close
Serialization.app.withValue(app) {
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val readA = in.readObject
@ -284,14 +284,14 @@ class ActorRefSpec extends AkkaSpec {
val baos = new ByteArrayOutputStream(8192 * 32)
val out = new ObjectOutputStream(baos)
val serialized = SerializedActorRef(app.address.hostname, app.address.port, "/this/path/does/not/exist")
val serialized = SerializedActorRef(system.root.remoteAddress.hostname, system.root.remoteAddress.port, "/this/path/does/not/exist")
out.writeObject(serialized)
out.flush
out.close
Serialization.app.withValue(app) {
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
(intercept[java.lang.IllegalStateException] {
in.readObject

View file

@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
}
}, timeout = t))
val defaultTimeout = app.AkkaConfig.ActorTimeout.duration
val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
val defaultTimeout = system.AkkaConfig.ActorTimeout.duration
val testTimeout = if (system.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
"An Actor-based Future" must {

View file

@ -9,7 +9,7 @@ class ClusterSpec extends AkkaSpec {
"be able to parse 'akka.actor.cluster._' config elements" in {
// TODO: make it use its own special config?
val config = app.config
val config = system.AkkaConfig.config
import config._
//akka.cluster

View file

@ -14,7 +14,7 @@ class DeployerSpec extends AkkaSpec {
"A Deployer" must {
"be able to parse 'akka.actor.deployment._' config elements" in {
val deployment = app.provider.deployer.lookupInConfig("/app/service-ping")
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookupInConfig("/app/service-ping")
deployment must be('defined)
deployment must equal(Some(

View file

@ -167,12 +167,12 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
}
})
filterException[Logging.EventHandlerException] {
app.eventStream.subscribe(testActor, classOf[Logging.Error])
system.eventStream.subscribe(testActor, classOf[Logging.Error])
fsm ! "go"
expectMsgPF(1 second, hint = "Next state 2 does not exist") {
case Logging.Error(_, `fsm`, "Next state 2 does not exist") true
}
app.eventStream.unsubscribe(testActor)
system.eventStream.unsubscribe(testActor)
}
}
@ -213,20 +213,20 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
case StopEvent(r, _, _) testActor ! r
}
})
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
fsm ! "go"
expectMsgPF(1 second, hint = "processing Event(go,null)") {
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[" + app.address + "/sys/testActor") true
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[") true
}
expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown"))
expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2"))
fsm ! "stop"
expectMsgPF(1 second, hint = "processing Event(stop,null)") {
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[" + app.address + "/sys/testActor") true
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[") true
}
expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal)
expectNoMsg(1 second)
app.eventStream.unsubscribe(testActor)
system.eventStream.unsubscribe(testActor)
}
}
}

View file

@ -37,14 +37,14 @@ class ForwardActorSpec extends AkkaSpec {
val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } })
val chain = createForwardingChain(app)
val chain = createForwardingChain(system)
chain.tell(ExpectedMessage, replyTo)
latch.await(Duration(5, "s")) must be === true
}
"forward actor reference when invoking forward on bang bang" in {
val chain = createForwardingChain(app)
val chain = createForwardingChain(system)
chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage
}
}

View file

@ -13,14 +13,15 @@ class LocalActorRefProviderSpec extends AkkaSpec {
"An LocalActorRefProvider" must {
"only create one instance of an actor with a specific address in a concurrent environment" in {
val provider = app.provider
val impl = system.asInstanceOf[ActorSystemImpl]
val provider = impl.provider
provider.isInstanceOf[LocalActorRefProvider] must be(true)
(0 until 100) foreach { i // 100 concurrent runs
val address = "new-actor" + i
implicit val timeout = Timeout(5 seconds)
((1 to 4) map { _ Future { provider.actorOf(Props(c { case _ }), app.guardian, address) } }).map(_.get).distinct.size must be(1)
((1 to 4) map { _ Future { provider.actorOf(impl, Props(c { case _ }), impl.guardian, address) } }).map(_.get).distinct.size must be(1)
}
}
}

View file

@ -53,7 +53,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"decorate a Receive" in {
new TestKit(appLogging) {
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val r: Actor.Receive = {
case null
}
@ -66,8 +66,8 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"be added on Actor if requested" in {
new TestKit(appLogging) with ImplicitSender {
ignoreMute(this)
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
app.eventStream.subscribe(testActor, classOf[Logging.Error])
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Error])
val actor = TestActorRef(new Actor {
def receive = loggable(this) {
case _ sender ! "x"
@ -95,7 +95,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"not duplicate logging" in {
new TestKit(appLogging) with ImplicitSender {
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor {
def receive = loggable(this)(loggable(this) {
case _ sender ! "x"
@ -115,7 +115,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"log AutoReceiveMessages if requested" in {
new TestKit(appAuto) {
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor {
def receive = {
case _
@ -137,10 +137,10 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val s = ref.toString
s.contains("MainBusReaper") || s.contains("Supervisor")
}
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
app.eventStream.subscribe(testActor, classOf[Logging.Error])
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Error])
within(3 seconds) {
val lifecycleGuardian = appLifecycle.guardian
val lifecycleGuardian = appLifecycle.asInstanceOf[ActorSystemImpl].guardian
val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
val supervisorSet = receiveWhile(messages = 2) {

View file

@ -16,7 +16,7 @@ import akka.testkit.AkkaSpec
class RestartStrategySpec extends AkkaSpec {
override def atStartup {
app.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
}
object Ping

View file

@ -28,14 +28,14 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
def receive = { case Tick countDownLatch.countDown() }
})
// run every 50 millisec
collectCancellable(app.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS))
collectCancellable(system.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS))
// after max 1 second it should be executed at least the 3 times already
assert(countDownLatch.await(1, TimeUnit.SECONDS))
val countDownLatch2 = new CountDownLatch(3)
collectCancellable(app.scheduler.schedule(() countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS))
collectCancellable(system.scheduler.schedule(() countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS))
// after max 1 second it should be executed at least the 3 times already
assert(countDownLatch2.await(2, TimeUnit.SECONDS))
@ -49,8 +49,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
})
// run every 50 millisec
collectCancellable(app.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
collectCancellable(app.scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
collectCancellable(system.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
collectCancellable(system.scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
// after 1 second the wait should fail
assert(countDownLatch.await(2, TimeUnit.SECONDS) == false)
@ -86,7 +86,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
})
(1 to 10).foreach { i
val timeout = collectCancellable(app.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS))
val timeout = collectCancellable(system.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS))
timeout.cancel()
}
@ -114,10 +114,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
})
val actor = (supervisor ? props).as[ActorRef].get
collectCancellable(app.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS))
collectCancellable(system.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS))
// appx 2 pings before crash
EventFilter[Exception]("CRASH", occurrences = 1) intercept {
collectCancellable(app.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS))
collectCancellable(system.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS))
}
assert(restartLatch.tryAwait(2, TimeUnit.SECONDS))

View file

@ -27,13 +27,13 @@ class SupervisorMiscSpec extends AkkaSpec {
}
})
val actor1 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor1 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor2 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor2 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor3 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newDispatcher("test").build)).as[ActorRef].get
val actor3 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newDispatcher("test").build)).as[ActorRef].get
val actor4 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor4 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
actor1 ! Kill
actor2 ! Kill

View file

@ -121,7 +121,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
}
override def atStartup() {
app.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage)))
system.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage)))
}
override def beforeEach() = {

View file

@ -147,18 +147,18 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
newFooBar(Props().withTimeout(Timeout(d)))
def newFooBar(props: Props): Foo =
app.typedActorOf(classOf[Foo], classOf[Bar], props)
system.typedActorOf(classOf[Foo], classOf[Bar], props)
def newStacked(props: Props = Props().withTimeout(Timeout(2000))): Stacked =
app.typedActorOf(classOf[Stacked], classOf[StackedImpl], props)
system.typedActorOf(classOf[Stacked], classOf[StackedImpl], props)
def mustStop(typedActor: AnyRef) = app.typedActor.stop(typedActor) must be(true)
def mustStop(typedActor: AnyRef) = system.typedActor.stop(typedActor) must be(true)
"TypedActors" must {
"be able to instantiate" in {
val t = newFooBar
app.typedActor.isTypedActor(t) must be(true)
system.typedActor.isTypedActor(t) must be(true)
mustStop(t)
}
@ -168,7 +168,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
}
"not stop non-started ones" in {
app.typedActor.stop(null) must be(false)
system.typedActor.stop(null) must be(false)
}
"throw an IllegalStateExcpetion when TypedActor.self is called in the wrong scope" in {
@ -187,7 +187,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to call toString" in {
val t = newFooBar
t.toString must be(app.typedActor.getActorRefFor(t).toString)
t.toString must be(system.typedActor.getActorRefFor(t).toString)
mustStop(t)
}
@ -200,7 +200,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to call hashCode" in {
val t = newFooBar
t.hashCode must be(app.typedActor.getActorRefFor(t).hashCode)
t.hashCode must be(system.typedActor.getActorRefFor(t).hashCode)
mustStop(t)
}
@ -295,7 +295,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
}
"be able to support implementation only typed actors" in {
val t = app.typedActorOf[Foo, Bar](Props())
val t = system.typedActorOf[Foo, Bar](Props())
val f = t.futurePigdog(200)
val f2 = t.futurePigdog(0)
f2.isCompleted must be(false)
@ -305,7 +305,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
}
"be able to support implementation only typed actors with complex interfaces" in {
val t = app.typedActorOf[Stackable1 with Stackable2, StackedImpl]()
val t = system.typedActorOf[Stackable1 with Stackable2, StackedImpl]()
t.stackable1 must be("foo")
t.stackable2 must be("bar")
mustStop(t)
@ -314,7 +314,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to use work-stealing dispatcher" in {
val props = Props(
timeout = Timeout(6600),
dispatcher = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
dispatcher = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(60)
.setMaxPoolSize(60)
@ -332,7 +332,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to serialize and deserialize invocations" in {
import java.io._
val m = TypedActor.MethodCall(app.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos)
@ -341,7 +341,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
Serialization.app.withValue(app) {
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method)
@ -351,7 +351,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to serialize and deserialize invocations' parameters" in {
import java.io._
val someFoo: Foo = new Bar
val m = TypedActor.MethodCall(app.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos)
@ -360,7 +360,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
Serialization.app.withValue(app) {
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method)

View file

@ -148,14 +148,14 @@ object ActorModelSpec {
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
starts: Long = dispatcher.starts.get(),
stops: Long = dispatcher.stops.get())(implicit app: ActorSystem) {
stops: Long = dispatcher.stops.get())(implicit system: ActorSystem) {
val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
try {
await(deadline)(starts == dispatcher.starts.get)
await(deadline)(stops == dispatcher.stops.get)
} catch {
case e
app.eventStream.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
system.eventStream.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
" required: starts=" + starts + ",stops=" + stops))
throw e
}
@ -181,7 +181,7 @@ object ActorModelSpec {
unregisters: Long = 0,
msgsReceived: Long = 0,
msgsProcessed: Long = 0,
restarts: Long = 0)(implicit app: ActorSystem) {
restarts: Long = 0)(implicit system: ActorSystem) {
assertRef(actorRef, dispatcher)(
suspensions,
resumes,
@ -199,7 +199,7 @@ object ActorModelSpec {
unregisters: Long = statsFor(actorRef).unregisters.get(),
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: ActorSystem) {
restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val deadline = System.currentTimeMillis + 1000
try {
@ -212,7 +212,7 @@ object ActorModelSpec {
await(deadline)(stats.restarts.get() == restarts)
} catch {
case e
app.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
system.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
throw e
@ -235,7 +235,7 @@ abstract class ActorModelSpec extends AkkaSpec {
import ActorModelSpec._
def newTestActor(dispatcher: MessageDispatcher) = app.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
def newTestActor(dispatcher: MessageDispatcher) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
protected def newInterceptedDispatcher: MessageDispatcherInterceptor
protected def dispatcherType: String
@ -318,7 +318,7 @@ abstract class ActorModelSpec extends AkkaSpec {
try {
f
} catch {
case e app.eventStream.publish(Error(e, this, "error in spawned thread"))
case e system.eventStream.publish(Error(e, this, "error in spawned thread"))
}
}
}
@ -421,10 +421,10 @@ class DispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", app.AkkaConfig.DispatcherThroughput,
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.AkkaConfig.DispatcherThroughput,
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Dispatcher"
@ -458,14 +458,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", 1, // TODO check why 1 here? (came from old test)
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
new BalancingDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", 1, // TODO check why 1 here? (came from old test)
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Balancing Dispatcher"
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = app.dispatcher
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = system.dispatcher
"A " + dispatcherType must {
"process messages in parallel" in {

View file

@ -8,7 +8,7 @@ import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class BalancingDispatcherSpec extends AkkaSpec {
def newWorkStealer() = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build
def newWorkStealer() = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build
val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer()

View file

@ -33,22 +33,22 @@ class DispatcherActorSpec extends AkkaSpec {
"A Dispatcher and an Actor" must {
"support tell" in {
val actor = actorOf(Props[OneWayTestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build))
val actor = actorOf(Props[OneWayTestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
val result = actor ! "OneWay"
assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
actor.stop()
}
"support ask/reply" in {
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build))
val actor = actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
val result = (actor ? "Hello").as[String]
assert("World" === result.get)
actor.stop()
}
"respect the throughput setting" in {
val throughputDispatcher = app.dispatcherFactory.
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType).
val throughputDispatcher = system.dispatcherFactory.
newDispatcher("THROUGHPUT", 101, 0, system.dispatcherFactory.MailboxType).
setCorePoolSize(1).
build
@ -76,8 +76,8 @@ class DispatcherActorSpec extends AkkaSpec {
"respect throughput deadline" in {
val deadlineMs = 100
val throughputDispatcher = app.dispatcherFactory.
newDispatcher("THROUGHPUT", 2, deadlineMs, app.dispatcherFactory.MailboxType).
val throughputDispatcher = system.dispatcherFactory.
newDispatcher("THROUGHPUT", 2, deadlineMs, system.dispatcherFactory.MailboxType).
setCorePoolSize(1).
build
val works = new AtomicBoolean(true)

View file

@ -12,7 +12,8 @@ import akka.config.Configuration
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class DispatchersSpec extends AkkaSpec {
import app.dispatcherFactory._
val df = system.dispatcherFactory
import df._
val tipe = "type"
val keepalivems = "keep-alive-time"

View file

@ -27,14 +27,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
"support tell" in {
var oneWay = new CountDownLatch(1)
val actor = actorOf(Props(self { case "OneWay" oneWay.countDown() }).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test")))
val actor = actorOf(Props(self { case "OneWay" oneWay.countDown() }).withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
val result = actor ! "OneWay"
assert(oneWay.await(1, TimeUnit.SECONDS))
actor.stop()
}
"support ask/reply" in {
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test")))
val actor = actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
val result = (actor ? "Hello").as[String]
assert("World" === result.get)
actor.stop()

View file

@ -13,7 +13,7 @@ class ConfigSpec extends AkkaSpec(ActorSystem("ConfigSpec", Configuration.fromFi
"The default configuration file (i.e. akka-reference.conf)" must {
"contain all configuration properties for akka-actor that are used in code with their correct defaults" in {
val config = app.config
val config = system.AkkaConfig.config
import config._
getList("akka.boot") must equal(Nil)

View file

@ -19,7 +19,7 @@ class Future2ActorSpec extends AkkaSpec {
}
"support reply via sender" in {
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
def receive = {
case "do" Future(31) pipeTo context.sender
case "ex" Future(throw new AssertionError) pipeTo context.sender

View file

@ -773,7 +773,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"ticket812FutureDispatchCleanup" in {
filterException[FutureTimeoutException] {
implicit val dispatcher = app.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build
implicit val dispatcher = system.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build
assert(dispatcher.tasks === 0)
val future = Future({ Thread.sleep(100); "Done" }, 10)
intercept[FutureTimeoutException] { future.await }

View file

@ -80,7 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
result
}
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, app.deadLetters)
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)
def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
q must not be null

View file

@ -18,12 +18,12 @@ class PriorityDispatcherSpec extends AkkaSpec {
testOrdering(BoundedPriorityMailbox(PriorityGenerator({
case i: Int i //Reverse order
case 'Result Int.MaxValue
}: Any Int), 1000, app.AkkaConfig.MailboxPushTimeout))
}: Any Int), 1000, system.AkkaConfig.MailboxPushTimeout))
}
}
def testOrdering(mboxType: MailboxType) {
val dispatcher = app.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
val dispatcher = system.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
val actor = actorOf(Props(new Actor {
var acc: List[Int] = Nil

View file

@ -6,7 +6,7 @@ package akka.event
import akka.testkit.AkkaSpec
import akka.config.Configuration
import akka.util.duration._
import akka.actor.{ Actor, ActorRef }
import akka.actor.{ Actor, ActorRef, ActorSystemImpl }
object EventStreamSpec {
case class M(i: Int)
@ -14,7 +14,7 @@ object EventStreamSpec {
case class SetTarget(ref: ActorRef)
class MyLog extends Actor {
var dst: ActorRef = app.deadLetters
var dst: ActorRef = system.deadLetters
def receive = {
case Logging.InitializeLogger(bus) bus.subscribe(context.self, classOf[SetTarget]); sender ! Logging.LoggerInitialized
case SetTarget(ref) dst = ref; dst ! "OK"
@ -36,11 +36,13 @@ class EventStreamSpec extends AkkaSpec(Configuration(
import EventStreamSpec._
val impl = system.asInstanceOf[ActorSystemImpl]
"An EventStream" must {
"manage subscriptions" in {
val bus = new EventStream(true)
bus.start(app.provider)
bus.start(impl)
bus.subscribe(testActor, classOf[M])
bus.publish(M(42))
within(1 second) {
@ -53,8 +55,8 @@ class EventStreamSpec extends AkkaSpec(Configuration(
"manage log levels" in {
val bus = new EventStream(false)
bus.start(app.provider)
bus.startDefaultLoggers(app.provider, app.AkkaConfig)
bus.start(impl)
bus.startDefaultLoggers(impl)
bus.publish(SetTarget(testActor))
expectMsg("OK")
within(2 seconds) {
@ -75,7 +77,7 @@ class EventStreamSpec extends AkkaSpec(Configuration(
val b2 = new B2
val c = new C
val bus = new EventStream(false)
bus.start(app.provider)
bus.start(impl)
within(2 seconds) {
bus.subscribe(testActor, classOf[B2]) === true
bus.publish(c)

View file

@ -17,7 +17,7 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic
class TellLatencyPerformanceSpec extends PerformanceSpec {
import TellLatencyPerformanceSpec._
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(8)
.build
@ -62,13 +62,13 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val clients = (for (i 0 until numberOfClients) yield {
val destination = app.actorOf[Destination]
val w4 = app.actorOf(new Waypoint(destination))
val w3 = app.actorOf(new Waypoint(w4))
val w2 = app.actorOf(new Waypoint(w3))
val w1 = app.actorOf(new Waypoint(w2))
val destination = system.actorOf[Destination]
val w4 = system.actorOf(new Waypoint(destination))
val w3 = system.actorOf(new Waypoint(w4))
val w2 = system.actorOf(new Waypoint(w3))
val w1 = system.actorOf(new Waypoint(w2))
Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher)
}).toList.map(app.actorOf(_))
}).toList.map(system.actorOf(_))
val start = System.nanoTime
clients.foreach(_ ! Run)

View file

@ -18,12 +18,12 @@ import akka.dispatch.Dispatchers
class TellThroughputPerformanceSpec extends PerformanceSpec {
import TellThroughputPerformanceSpec._
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
val destinationDispatcher = app.dispatcherFactory.newDispatcher("destination-dispatcher")
val destinationDispatcher = system.dispatcherFactory.newDispatcher("destination-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
@ -71,9 +71,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients)
yield app.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
yield system.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
val clients = for (dest destinations)
yield app.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
val start = System.nanoTime
clients.foreach(_ ! Run)

View file

@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TradingLatencyPerformanceSpec extends PerformanceSpec {
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
@ -38,7 +38,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
override def beforeEach() {
super.beforeEach()
stat = new SynchronizedDescriptiveStatistics
tradingSystem = new AkkaTradingSystem(app)
tradingSystem = new AkkaTradingSystem(system)
tradingSystem.start()
TotalTradeCounter.reset()
stat = new SynchronizedDescriptiveStatistics
@ -99,7 +99,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher)
app.actorOf(props)
system.actorOf(props)
})
clients.foreach(_ ! "run")

View file

@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TradingThroughputPerformanceSpec extends PerformanceSpec {
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
@ -30,7 +30,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
override def beforeEach() {
super.beforeEach()
tradingSystem = new AkkaTradingSystem(app)
tradingSystem = new AkkaTradingSystem(system)
tradingSystem.start()
TotalTradeCounter.reset()
}
@ -92,7 +92,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher)
app.actorOf(props)
system.actorOf(props)
})
clients.foreach(_ ! "run")

View file

@ -11,8 +11,6 @@ import akka.actor.ActorSystem
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
def app: ActorSystem
def isBenchmark() = System.getProperty("benchmark") == "true"
def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
@ -29,7 +27,7 @@ trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
}
val resultRepository = BenchResultRepository()
lazy val report = new Report(app, resultRepository, compareResultWith)
lazy val report = new Report(system, resultRepository, compareResultWith)
/**
* To compare two tests with each other you can override this method, in

View file

@ -224,8 +224,8 @@ class Report(
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
sb.append("\n")
sb.append("Akka config:")
for (key app.config.keys) {
sb.append("\n ").append(key).append("=").append(app.config(key))
for (key app.AkkaConfig.config.keys) {
sb.append("\n ").append(key).append("=").append(app.AkkaConfig.config(key))
}
sb.toString

View file

@ -329,7 +329,7 @@ class ActorPoolSpec extends AkkaSpec {
}
"support typed actors" in {
val pool = app.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
val pool = system.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
def lowerBound = 1
def upperBound = 5
def pressureThreshold = 1
@ -338,7 +338,7 @@ class ActorPoolSpec extends AkkaSpec {
def rampupRate = 0.1
def backoffRate = 0.50
def backoffThreshold = 0.50
def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
def instance(p: Props) = system.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
def receive = _route
}, Props().withTimeout(10 seconds).withFaultHandler(faultHandler))
@ -348,7 +348,7 @@ class ActorPoolSpec extends AkkaSpec {
val value = r.get
value must equal(i * i)
}
app.typedActor.stop(pool)
system.typedActor.stop(pool)
}
"provide default supervision of pooled actors" in {

View file

@ -11,12 +11,14 @@ import akka.routing.Routing.Broadcast
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfiguredLocalRoutingSpec extends AkkaSpec {
val deployer = system.asInstanceOf[ActorSystemImpl].provider.deployer
"round robin router" must {
"be able to shut down its instance" in {
val path = app / "round-robin-0"
val path = system / "round-robin-0"
app.provider.deployer.deploy(
deployer.deploy(
Deploy(
path.toString,
None,
@ -27,7 +29,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5)
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
def receive = {
case "hello" helloLatch.countDown()
}
@ -49,9 +51,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
}
"deliver messages in a round robin fashion" in {
val path = app / "round-robin-1"
val path = system / "round-robin-1"
app.provider.deployer.deploy(
deployer.deploy(
Deploy(
path.toString,
None,
@ -69,7 +71,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
replies = replies + (i -> 0)
}
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement()
def receive = {
case "hit" sender ! id
@ -93,9 +95,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
}
"deliver a broadcast message using the !" in {
val path = app / "round-robin-2"
val path = system / "round-robin-2"
app.provider.deployer.deploy(
deployer.deploy(
Deploy(
path.toString,
None,
@ -106,7 +108,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5)
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
def receive = {
case "hello" helloLatch.countDown()
}
@ -127,9 +129,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"random router" must {
"be able to shut down its instance" in {
val path = app / "random-0"
val path = system / "random-0"
app.provider.deployer.deploy(
deployer.deploy(
Deploy(
path.toString,
None,
@ -139,7 +141,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val stopLatch = new CountDownLatch(7)
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
def receive = {
case "hello" {}
}
@ -160,9 +162,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
}
"deliver messages in a random fashion" in {
val path = app / "random-1"
val path = system / "random-1"
app.provider.deployer.deploy(
deployer.deploy(
Deploy(
path.toString,
None,
@ -180,7 +182,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
replies = replies + (i -> 0)
}
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement()
def receive = {
case "hit" sender ! id
@ -204,9 +206,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
}
"deliver a broadcast message using the !" in {
val path = app / "random-2"
val path = system / "random-2"
app.provider.deployer.deploy(
deployer.deploy(
Deploy(
path.toString,
None,
@ -217,7 +219,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(6)
val stopLatch = new CountDownLatch(6)
val actor = app.actorOf(Props(new Actor {
val actor = system.actorOf(Props(new Actor {
def receive = {
case "hello" helloLatch.countDown()
}

View file

@ -22,6 +22,8 @@ object RoutingSpec {
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class RoutingSpec extends AkkaSpec {
val impl = system.asInstanceOf[ActorSystemImpl]
import akka.routing.RoutingSpec._
"direct router" must {
@ -29,7 +31,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false)
}
@ -45,7 +47,7 @@ class RoutingSpec extends AkkaSpec {
})
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val routedActor = new RoutedActorRef(app, props, app.guardian, "foo")
val routedActor = new RoutedActorRef(system, props, impl.guardian, "foo")
routedActor ! "hello"
routedActor ! "end"
@ -66,7 +68,7 @@ class RoutingSpec extends AkkaSpec {
})
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1)
actor ! "end"
@ -83,7 +85,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false)
}
@ -113,7 +115,7 @@ class RoutingSpec extends AkkaSpec {
//create the routed actor.
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(connections))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
//send messages to the actor.
for (i 0 until iterationCount) {
@ -152,7 +154,7 @@ class RoutingSpec extends AkkaSpec {
})
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -175,7 +177,7 @@ class RoutingSpec extends AkkaSpec {
})
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
intercept[RoutingException] { actor ? Broadcast(1) }
@ -192,7 +194,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false)
}
@ -216,7 +218,7 @@ class RoutingSpec extends AkkaSpec {
})
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -239,7 +241,7 @@ class RoutingSpec extends AkkaSpec {
})
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
try {
actor ? Broadcast(1)
@ -262,7 +264,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(Stop(Some(0)))
@ -277,7 +279,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(Stop())
@ -293,7 +295,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
(actor ? Broadcast("Hi!")).get.asInstanceOf[Int] must be(0)
@ -302,14 +304,14 @@ class RoutingSpec extends AkkaSpec {
"return the first response from connections, when some of them failed to reply" in {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"be started when constructed" in {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false)
}
@ -324,7 +326,7 @@ class RoutingSpec extends AkkaSpec {
for (i 0 until connectionCount) {
counters = counters :+ new AtomicInteger()
val connection = app.actorOf(new Actor {
val connection = system.actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg)
@ -335,7 +337,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(connections))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
for (i 0 until iterationCount) {
for (k 0 until connectionCount) {
@ -357,7 +359,7 @@ class RoutingSpec extends AkkaSpec {
val doneLatch = new TestLatch(2)
val counter1 = new AtomicInteger
val connection1 = app.actorOf(new Actor {
val connection1 = system.actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
@ -365,7 +367,7 @@ class RoutingSpec extends AkkaSpec {
})
val counter2 = new AtomicInteger
val connection2 = app.actorOf(new Actor {
val connection2 = system.actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
@ -374,7 +376,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -387,7 +389,7 @@ class RoutingSpec extends AkkaSpec {
case class Stop(id: Option[Int] = None)
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = app.actorOf(new Actor {
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(new Actor {
def receive = {
case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) self.stop()

View file

@ -7,7 +7,7 @@ package akka.serialization
import akka.serialization.Serialization._
import scala.reflect._
import akka.testkit.AkkaSpec
import akka.actor.ActorSystem
import akka.actor.{ ActorSystem, ActorSystemImpl }
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import akka.actor.DeadLetterActorRef
@ -24,7 +24,8 @@ object SerializeSpec {
class SerializeSpec extends AkkaSpec {
import SerializeSpec._
import app.serialization._
val ser = system.serialization
import ser._
"Serialization" must {
@ -68,13 +69,13 @@ class SerializeSpec extends AkkaSpec {
"serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf)
val a = new ActorSystem()
val a = ActorSystem()
out.writeObject(a.deadLetters)
out.flush()
out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
Serialization.app.withValue(a) {
Serialization.app.withValue(a.asInstanceOf[ActorSystemImpl]) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) must be(true)
}

View file

@ -11,7 +11,7 @@ import org.junit.{ After, Test }
class CallingThreadDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "test") with MessageDispatcherInterceptor
def newInterceptedDispatcher = new CallingThreadDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "test") with MessageDispatcherInterceptor
def dispatcherType = "Calling Thread Dispatcher"
}

View file

@ -154,7 +154,7 @@ object Timeout {
}
trait ActorLogging { this: Actor
val log = akka.event.Logging(app.eventStream, context.self)
val log = akka.event.Logging(system.eventStream, context.self)
}
object Actor {
@ -229,12 +229,12 @@ trait Actor {
context
}
implicit def app = context.app
implicit def system = context.system
/**
* The default timeout, based on the config setting 'akka.actor.timeout'
*/
implicit def defaultTimeout = app.AkkaConfig.ActorTimeout
implicit def defaultTimeout = system.AkkaConfig.ActorTimeout
/**
* Wrap a Receive partial function in a logging enclosure, which sends a
@ -250,7 +250,7 @@ trait Actor {
* This method does NOT modify the given Receive unless
* akka.actor.debug.receive is set within akka.conf.
*/
def loggable(self: AnyRef)(r: Receive): Receive = if (app.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
def loggable(self: AnyRef)(r: Receive): Receive = if (system.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
/**
* Some[ActorRef] representation of the 'self' ActorRef reference.

View file

@ -45,7 +45,7 @@ trait ActorContext extends ActorRefFactory with TypedActorFactory {
def handleChildTerminated(child: ActorRef): Unit
def app: ActorSystem
def system: ActorSystem
def parent: ActorRef
}
@ -63,7 +63,7 @@ private[akka] object ActorCell {
//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)
private[akka] class ActorCell(
val app: ActorSystem,
val app: ActorSystemImpl,
val self: ActorRef with ScalaActorRef,
val props: Props,
val parent: ActorRef,
@ -72,6 +72,8 @@ private[akka] class ActorCell(
import ActorCell._
final def system = app
protected final def guardian = self
protected def typedActor = app.typedActor

View file

@ -4,6 +4,8 @@
package akka.actor
import akka.remote.RemoteAddress
object ActorPath {
final val separator = "/"
@ -58,9 +60,9 @@ object ActorPath {
*/
trait ActorPath {
/**
* The akka application for this path.
* The RemoteAddress for this path.
*/
def app: ActorSystem
def remoteAddress: RemoteAddress
/**
* The name of the actor that this path refers to.
@ -78,9 +80,9 @@ trait ActorPath {
def /(child: String): ActorPath
/**
* Find the ActorRef for this path.
* Recursively create a descendants path by appending all child names.
*/
def ref: Option[ActorRef]
def /(child: Iterable[String]): ActorPath = (this /: child)(_ / _)
/**
* String representation of this path. Different from toString for root path.
@ -98,15 +100,13 @@ trait ActorPath {
def isRoot: Boolean
}
class RootActorPath(val app: ActorSystem) extends ActorPath {
class RootActorPath(val remoteAddress: RemoteAddress) extends ActorPath {
def name: String = "/"
def parent: ActorPath = this
def /(child: String): ActorPath = new ChildActorPath(app, this, child)
def ref: Option[ActorRef] = app.actorFor(path)
def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
def string: String = ""
@ -117,11 +117,9 @@ class RootActorPath(val app: ActorSystem) extends ActorPath {
override def toString = ActorPath.separator
}
class ChildActorPath(val app: ActorSystem, val parent: ActorPath, val name: String) extends ActorPath {
class ChildActorPath(val remoteAddress: RemoteAddress, val parent: ActorPath, val name: String) extends ActorPath {
def /(child: String): ActorPath = new ChildActorPath(app, this, child)
def ref: Option[ActorRef] = app.actorFor(path)
def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
def string: String = parent.string + ActorPath.separator + name

View file

@ -13,6 +13,7 @@ import java.net.InetSocketAddress
import akka.remote.RemoteAddress
import java.util.concurrent.TimeUnit
import akka.event.EventStream
import akka.event.DeathWatch
/**
* ActorRef is an immutable and serializable handle to an Actor.
@ -161,7 +162,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class LocalActorRef private[akka] (
app: ActorSystem,
app: ActorSystemImpl,
_props: Props,
_supervisor: ActorRef,
val path: ActorPath,
@ -172,7 +173,7 @@ class LocalActorRef private[akka] (
def name = path.name
def address: String = app.address + path.toString
def address: String = path.toString
/*
* actorCell.start() publishes actorCell & this to the dispatcher, which
@ -415,16 +416,15 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) exte
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
}
abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef {
abstract class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deathWatch: DeathWatch, timeout: Timeout, val dispatcher: MessageDispatcher) extends MinimalActorRef {
final val result = new DefaultPromise[Any](timeout)(dispatcher)
// FIXME (actor path): put this under the tmp guardian supervisor
val path: ActorPath = app.root / "tmp" / name
override def name = path.name
def address: String = app.address + path.toString
def address: String = path.toString
{
val callback: Future[Any] Unit = { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
val callback: Future[Any] Unit = { _ deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
result onComplete callback
result onTimeout callback
}
@ -450,5 +450,5 @@ abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = ap
override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped"))
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = app.provider.serialize(this)
private def writeReplace(): AnyRef = provider.serialize(this)
}

View file

@ -22,7 +22,7 @@ import com.eaio.uuid.UUID
*/
trait ActorRefProvider {
def actorOf(props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(props, supervisor, name, false)
def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(app, props, supervisor, name, false)
def actorFor(path: Iterable[String]): Option[ActorRef]
@ -36,6 +36,8 @@ trait ActorRefProvider {
def AkkaConfig: ActorSystem.AkkaConfig
def init(app: ActorSystemImpl)
/**
* What deployer will be used to resolve deployment configuration?
*/
@ -43,9 +45,9 @@ trait ActorRefProvider {
private[akka] def scheduler: Scheduler
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
private[akka] def evict(path: String): Boolean
@ -61,7 +63,7 @@ trait ActorRefProvider {
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
private[akka] def tempPath: String
private[akka] def tempName: String
}
/**
@ -69,9 +71,11 @@ trait ActorRefProvider {
*/
trait ActorRefFactory {
def provider: ActorRefProvider
protected def app: ActorSystemImpl
def dispatcher: MessageDispatcher
protected def provider: ActorRefProvider
protected def dispatcher: MessageDispatcher
/**
* Father of all children created by this interface.
@ -85,7 +89,7 @@ trait ActorRefFactory {
Helpers.base64(l)
}
def actorOf(props: Props): ActorRef = provider.actorOf(props, guardian, randomName, false)
def actorOf(props: Props): ActorRef = provider.actorOf(app, props, guardian, randomName, false)
/*
* TODO this will have to go at some point, because creating two actors with
@ -95,7 +99,7 @@ trait ActorRefFactory {
def actorOf(props: Props, name: String): ActorRef = {
if (name == null || name == "" || name.startsWith("$"))
throw new ActorInitializationException("actor name must not be null, empty or start with $")
provider.actorOf(props, guardian, name, false)
provider.actorOf(app, props, guardian, name, false)
}
def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]))
@ -109,6 +113,8 @@ trait ActorRefFactory {
def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() creator.create()))
def actorFor(path: ActorPath): Option[ActorRef] = actorFor(path.path)
def actorFor(path: String): Option[ActorRef] = actorFor(ActorPath.split(path))
def actorFor(path: Iterable[String]): Option[ActorRef] = provider.actorFor(path)
@ -120,7 +126,6 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
* Local ActorRef provider.
*/
class LocalActorRefProvider(
private val app: ActorSystem,
val AkkaConfig: ActorSystem.AkkaConfig,
val root: ActorPath,
val eventStream: EventStream,
@ -136,16 +141,18 @@ class LocalActorRefProvider(
private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename)
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher)
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher)
/*
* generate name for temporary actor refs
*/
private val tempNumber = new AtomicLong
def tempPath = {
def tempName = {
val l = tempNumber.getAndIncrement()
"$_" + Helpers.base64(l)
}
private val tempNode = root / "tmp"
def tempPath = tempNode / tempName
// FIXME (actor path): this could become a cache for the new tree traversal actorFor
// currently still used for tmp actors (e.g. ask actor refs)
@ -159,12 +166,12 @@ class LocalActorRefProvider(
@volatile
var stopped = false
val name = app.name + "-bubble-walker"
val name = "bubble-walker"
// FIXME (actor path): move the root path to the new root guardian
val path = app.root
val path = root / name
val address = app.address + path.toString
val address = path.toString
override def toString = name
@ -208,20 +215,33 @@ class LocalActorRefProvider(
}
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
private val rootGuardian: ActorRef = actorOf(guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true)
val guardian: ActorRef = actorOf(guardianProps, rootGuardian, "app", true)
val systemGuardian: ActorRef = actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
/*
* The problem is that ActorRefs need a reference to the ActorSystem to
* provide their service. Hence they cannot be created while the
* constructors of ActorSystem and ActorRefProvider are still running.
* The solution is to split out that last part into an init() method,
* but it also requires these references to be @volatile.
*/
@volatile
private var rootGuardian: ActorRef = _
@volatile
var guardian: ActorRef = _
@volatile
var systemGuardian: ActorRef = _
val deathWatch = createDeathWatch()
def init(app: ActorSystemImpl) {
rootGuardian = actorOf(app, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true)
guardian = actorOf(app, guardianProps, rootGuardian, "app", true)
systemGuardian = actorOf(app, guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
// chain death watchers so that killing guardian stops the application
deathWatch.subscribe(systemGuardian, guardian)
deathWatch.subscribe(rootGuardian, systemGuardian)
}
// FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "app" name for now)
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(app.guardian), path.tail)
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(guardian), path.tail)
@tailrec
private def findInTree(start: Option[ActorRef], path: Iterable[String]): Option[ActorRef] = {
@ -246,12 +266,12 @@ class LocalActorRefProvider(
*/
private[akka] def evict(path: String): Boolean = actors.remove(path) ne null
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(app, props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
val name = path.name
val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout?
val newFuture = Promise[ActorRef](5000)(dispatcher) // FIXME is this proper timeout?
actors.putIfAbsent(path.toString, newFuture) match {
case null
@ -270,7 +290,7 @@ class LocalActorRefProvider(
case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()(
if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout)
if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, AkkaConfig.ActorTimeout)
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
@ -282,7 +302,7 @@ class LocalActorRefProvider(
new LocalActorRef(app, props, supervisor, routedPath, systemService)
}
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
case unknown throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown)
}
@ -307,7 +327,7 @@ class LocalActorRefProvider(
/**
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
*/
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
// FIXME: this needs to take supervision into account!
//FIXME clustering should be implemented by cluster actor ref provider
@ -324,16 +344,16 @@ class LocalActorRefProvider(
}
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path))
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(app.address, actor.path.toString)
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(root.remoteAddress, actor.path.toString)
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = {
import akka.dispatch.{ Future, Promise, DefaultPromise }
(if (within == null) app.AkkaConfig.ActorTimeout else within) match {
case t if t.duration.length <= 0 new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout
(if (within == null) AkkaConfig.ActorTimeout else within) match {
case t if t.duration.length <= 0 new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout
case t
val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this) }
val a = new AskActorRef(tempPath, this, deathWatch, t, dispatcher) { def whenDone() = actors.remove(this) }
assert(actors.putIfAbsent(a.path.toString, a) eq null) //If this fails, we're in deep trouble
recipient.tell(message, a)
a.result

View file

@ -61,11 +61,14 @@ object ActorSystem {
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
def apply(name: String, config: Configuration) = new ActorSystem(name, config)
def create(name: String, config: Configuration): ActorSystem = apply(name, config)
def apply(name: String, config: Configuration): ActorSystem = new ActorSystemImpl(name, config).start()
def apply(name: String): ActorSystem = new ActorSystem(name)
def create(name: String): ActorSystem = apply(name)
def apply(name: String): ActorSystem = apply(name, defaultConfig)
def apply(): ActorSystem = new ActorSystem()
def create(): ActorSystem = apply()
def apply(): ActorSystem = apply("default")
sealed trait ExitStatus
case object Stopped extends ExitStatus
@ -126,22 +129,54 @@ object ActorSystem {
}
class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory {
abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
import ActorSystem._
def this(name: String) = this(name, ActorSystem.defaultConfig)
def this() = this("default")
def name: String
def AkkaConfig: AkkaConfig
def nodename: String
val AkkaConfig = new AkkaConfig(config)
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true)
import AkkaConfig._
/**
* Construct a path below the application guardian.
*/
def /(name: String): ActorPath
def root: ActorPath
val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000
def eventStream: EventStream
def log: LoggingAdapter
def deadLetters: ActorRef
def deadLetterMailbox: Mailbox
// FIXME: Serialization should be an extension
def serialization: Serialization
// FIXME: TypedActor should be an extension
def typedActor: TypedActor
def scheduler: Scheduler
def dispatcherFactory: Dispatchers
def dispatcher: MessageDispatcher
def registerOnTermination(code: Unit)
def registerOnTermination(code: Runnable)
def stop()
}
class ActorSystemImpl(val name: String, config: Configuration) extends ActorSystem {
import ActorSystem._
val AkkaConfig = new AkkaConfig(config)
protected def app = this
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true)
import AkkaConfig._
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostAddress
case value value
@ -158,7 +193,7 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
/**
* The root actor path for this application.
*/
val root: ActorPath = new RootActorPath(this)
val root: ActorPath = new RootActorPath(address)
val deadLetters = new DeadLetterActorRef(eventStream, root / "nul")
val deadLetterMailbox = new Mailbox(null) {
@ -181,16 +216,21 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
deadLetters.init(dispatcher)
// TODO think about memory consistency effects when doing funky stuff inside constructor
val provider: ActorRefProvider = {
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
case Left(e) throw e
case Right(b) b
}
val params: Array[Class[_]] = Array(classOf[ActorSystem], classOf[AkkaConfig], classOf[ActorPath], classOf[EventStream], classOf[MessageDispatcher], classOf[Scheduler])
val args: Array[AnyRef] = Array(this, AkkaConfig, root, eventStream, dispatcher, scheduler)
val arguments = List(
classOf[AkkaConfig] -> AkkaConfig,
classOf[ActorPath] -> root,
classOf[EventStream] -> eventStream,
classOf[MessageDispatcher] -> dispatcher,
classOf[Scheduler] -> scheduler)
val types: Array[Class[_]] = arguments map (_._1) toArray
val values: Array[AnyRef] = arguments map (_._2) toArray
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, params, args) match {
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
case Left(e) throw e
case Right(p) p
}
@ -200,24 +240,33 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
def guardian: ActorRef = provider.guardian
def systemGuardian: ActorRef = provider.systemGuardian
def deathWatch: DeathWatch = provider.deathWatch
def nodename: String = provider.nodename
terminationFuture.onComplete(_ scheduler.stop())
terminationFuture.onComplete(_ dispatcher.shutdown())
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
eventStream.start(provider)
eventStream.startDefaultLoggers(provider, AkkaConfig)
@volatile
private var _serialization: Serialization = _
def serialization = _serialization
@volatile
private var _typedActor: TypedActor = _
def typedActor = _typedActor
// TODO think about memory consistency effects when doing funky stuff inside constructor
val serialization = new Serialization(this)
val typedActor = new TypedActor(AkkaConfig, serialization)
/**
* Create an actor path under the application supervisor (/app).
*/
def /(actorName: String): ActorPath = guardian.path / actorName
def start(): this.type = {
_serialization = new Serialization(this)
_typedActor = new TypedActor(AkkaConfig, _serialization)
provider.init(this)
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
eventStream.start(this)
eventStream.startDefaultLoggers(this)
this
}
def registerOnTermination(code: Unit) { terminationFuture onComplete (_ code) }
def registerOnTermination(code: Runnable) { terminationFuture onComplete (_ code.run) }
// TODO shutdown all that other stuff, whatever that may be
def stop() {
guardian.stop()

View file

@ -188,7 +188,7 @@ trait FSM[S, D] extends ListenerManagement {
type Timeout = Option[Duration]
type TransitionHandler = PartialFunction[(S, S), Unit]
val log = Logging(app.eventStream, context.self)
val log = Logging(system, context.self)
/**
* ****************************************
@ -522,7 +522,7 @@ trait FSM[S, D] extends ListenerManagement {
if (timeout.isDefined) {
val t = timeout.get
if (t.finite_? && t.length >= 0) {
timeoutFuture = Some(app.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
timeoutFuture = Some(system.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
}
}
}
@ -565,7 +565,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
def logDepth: Int = 0
private val debugEvent = context.app.AkkaConfig.FsmDebugEvent
private val debugEvent = system.AkkaConfig.FsmDebugEvent
private val events = new Array[Event](logDepth)
private val states = new Array[AnyRef](logDepth)

View file

@ -253,7 +253,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor {
var worker: IOWorker = _
override def preStart {
worker = new IOWorker(app, self, bufferSize)
worker = new IOWorker(system, self, bufferSize)
worker.start()
}

View file

@ -330,7 +330,7 @@ class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization)
def receive = {
case m: MethodCall
TypedActor.selfReference set proxyVar.get
TypedActor.appReference set app
TypedActor.appReference set system
try {
if (m.isOneWay) m(me)
else {

View file

@ -8,10 +8,6 @@ package object actor {
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
// actor path can be used as an actor ref (note: does a lookup in the app using path.ref)
implicit def actorPath2Ref(path: ActorPath): ActorRef = path.ref.getOrElse(path.app.deadLetters)
implicit def actorPath2ScalaRef(path: ActorPath): ScalaActorRef = actorPath2Ref(path).asInstanceOf[ScalaActorRef]
type Uuid = com.eaio.uuid.UUID
def newUuid(): Uuid = new Uuid()

View file

@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) {
*/
object NodeAddress {
def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName)
def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.provider.nodename)
def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename)
def unapply(other: Any) = other match {
case address: NodeAddress Some((address.clusterName, address.nodeName))

View file

@ -3,11 +3,8 @@
*/
package akka.event
import akka.actor.{ ActorRef, Actor, Props }
import akka.actor.ActorSystem
import akka.actor.Terminated
import akka.actor.{ ActorRef, Actor, Props, ActorSystemImpl, Terminated }
import akka.util.Subclassification
import akka.actor.ActorRefProvider
class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification {
@ -42,13 +39,13 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas
super.unsubscribe(subscriber)
}
def start(provider: ActorRefProvider) {
reaper = provider.actorOf(Props(new Actor {
def start(app: ActorSystemImpl) {
reaper = app.systemActorOf(Props(new Actor {
def receive = {
case ref: ActorRef watch(ref)
case Terminated(ref) unsubscribe(ref)
}
}), provider.systemGuardian, "MainBusReaper", true)
}), "MainBusReaper")
subscribers foreach (reaper ! _)
}

View file

@ -3,7 +3,7 @@
*/
package akka.event
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, simpleName }
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, ActorSystemImpl, simpleName }
import akka.AkkaException
import akka.actor.ActorSystem.AkkaConfig
import akka.util.ReflectiveAccess
@ -79,13 +79,13 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "StandardOutLogger started"))
}
private[akka] def startDefaultLoggers(provider: ActorRefProvider, config: AkkaConfig) {
val level = levelFor(config.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel))
private[akka] def startDefaultLoggers(app: ActorSystemImpl) {
val level = levelFor(app.AkkaConfig.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.AkkaConfig.LogLevel))
ErrorLevel
}
try {
val defaultLoggers = config.EventHandlers match {
val defaultLoggers = app.AkkaConfig.EventHandlers match {
case Nil "akka.event.Logging$DefaultLogger" :: Nil
case loggers loggers
}
@ -95,7 +95,7 @@ trait LoggingBus extends ActorEventBus {
} yield {
try {
ReflectiveAccess.getClassFor[Actor](loggerName) match {
case Right(actorClass) addLogger(provider, actorClass, level)
case Right(actorClass) addLogger(app, actorClass, level)
case Left(exception) throw exception
}
} catch {
@ -138,9 +138,9 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "all default loggers stopped"))
}
private def addLogger(provider: ActorRefProvider, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
private def addLogger(app: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz)
val actor = provider.actorOf(Props(clazz), provider.systemGuardian, name, true)
val actor = app.systemActorOf(Props(clazz), name)
implicit val timeout = Timeout(3 seconds)
val response = try actor ? InitializeLogger(this) get catch {
case _: FutureTimeoutException

View file

@ -6,7 +6,7 @@ package akka.serialization
import akka.AkkaException
import akka.util.ReflectiveAccess
import akka.actor.ActorSystem
import akka.actor.{ ActorSystem, ActorSystemImpl }
import scala.util.DynamicVariable
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
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
*/
class Serialization(val app: ActorSystem) {
class Serialization(val app: ActorSystemImpl) {
//TODO document me
def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
@ -70,7 +70,7 @@ class Serialization(val app: ActorSystem) {
* But "default" can be overridden in config
*/
val serializers: Map[String, Serializer] =
app.config.getSection("akka.actor.serializers")
app.AkkaConfig.config.getSection("akka.actor.serializers")
.map(_.map)
.getOrElse(Map())
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) {
@ -81,7 +81,7 @@ class Serialization(val app: ActorSystem) {
/**
* bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used
*/
val bindings: Map[String, String] = app.config.getSection("akka.actor.serialization-bindings") map {
val bindings: Map[String, String] = app.AkkaConfig.config.getSection("akka.actor.serialization-bindings") map {
_.map.foldLeft(Map[String, String]()) {
case (result, (k: String, vs: List[_])) result ++ (vs collect { case v: String (v, k) }) //All keys which are lists, take the Strings from them and Map them
case (result, _) result //For any other values, just skip them, TODO: print out warnings?
@ -102,6 +102,6 @@ class Serialization(val app: ActorSystem) {
object Serialization {
// TODO ensure that these are always set (i.e. withValue()) when doing deserialization
val app = new DynamicVariable[ActorSystem](null)
val app = new DynamicVariable[ActorSystemImpl](null)
}

View file

@ -14,7 +14,7 @@ import akka.config.Configuration
//#my-actor
class MyActor extends Actor {
val log = Logging(app, this)
val log = Logging(system, this)
def receive = {
case "test" log.info("received test")
case _ log.info("received unknown message")
@ -36,8 +36,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
case e: Logging.Info true
case _ false
}
app.eventStream.publish(TestEvent.Mute(filter))
app.eventStream.subscribe(testActor, classOf[Logging.Info])
system.eventStream.publish(TestEvent.Mute(filter))
system.eventStream.subscribe(testActor, classOf[Logging.Info])
myActor ! "test"
expectMsgPF(1 second) { case Logging.Info(_, "received test") true }
@ -45,8 +45,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
myActor ! "unknown"
expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true }
app.eventStream.unsubscribe(testActor)
app.eventStream.publish(TestEvent.UnMute(filter))
system.eventStream.unsubscribe(testActor)
system.eventStream.publish(TestEvent.UnMute(filter))
myActor.stop()
}

View file

@ -27,8 +27,8 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
def this(app: ActorSystem) {
this(
app.config.getInt("akka.remote.failure-detector.theshold", 8),
app.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
app.AkkaConfig.config.getInt("akka.remote.failure-detector.theshold", 8),
app.AkkaConfig.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
}
private final val PhiFactor = 1.0 / math.log(10.0)

View file

@ -107,7 +107,7 @@ class Gossiper(remote: Remote) {
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
private val seeds = Set(address) // FIXME read in list of seeds from config
private val address = app.address
private val address = app.root.remoteAddress
private val nodeFingerprint = address.##
private val random = SecureRandom.getInstance("SHA1PRNG")
@ -153,7 +153,7 @@ class Gossiper(remote: Remote) {
node oldAvailableNodes
if connectionManager.connectionFor(node).isEmpty
} {
val connectionFactory = () RemoteActorRef(remote.server, gossipingNode, remote.remoteDaemon.path, None)
val connectionFactory = () RemoteActorRef(remote.app.provider, remote.server, gossipingNode, remote.remoteDaemon.path, None)
connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes
}

View file

@ -7,7 +7,7 @@ package akka.remote
import scala.collection.mutable
import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid }
import akka.actor.Actor._
import akka.actor.ActorSystem
import akka.actor.ActorSystemImpl
/**
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
@ -58,12 +58,12 @@ object NetworkEventStream {
}
}
class NetworkEventStream(val app: ActorSystem) {
class NetworkEventStream(app: ActorSystemImpl) {
import NetworkEventStream._
// FIXME: check that this supervision is correct
private[akka] val sender = app.provider.actorOf(
private[akka] val sender = app.provider.actorOf(app,
Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
app.systemGuardian, "network-event-sender", systemService = true)

View file

@ -15,25 +15,24 @@ import akka.actor.DeploymentConfig._
import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress
import com.eaio.uuid.UUID
import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression }
import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
import java.util.concurrent.atomic.AtomicLong
/**
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Remote(val app: ActorSystem, val nodename: String) {
class Remote(val app: ActorSystemImpl, val nodename: String) {
val log = Logging(app, this)
import app._
import app.config
import app.AkkaConfig._
val AC = AkkaConfig
import AC._
// TODO move to AkkaConfig?
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
@ -48,11 +47,12 @@ class Remote(val app: ActorSystem, val nodename: String) {
// FIXME configure computeGridDispatcher to what?
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
// FIXME it is probably better to create another supervisor for handling the children created by handle_*
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want?
private[remote] lazy val remoteDaemon =
app.provider.actorOf(
app.provider.actorOf(app,
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
remoteDaemonSupervisor,
remoteDaemonServiceName,
@ -81,7 +81,7 @@ class Remote(val app: ActorSystem, val nodename: String) {
}
def start(): Unit = {
val serverAddress = server.app.address //Force init of server
val serverAddress = server.app.root.remoteAddress //Force init of server
val daemonAddress = remoteDaemon.address //Force init of daemon
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress)
}
@ -139,10 +139,10 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
}
val actorPath = ActorPath(remote.app, message.getActorPath)
val parent = actorPath.parent.ref
val parent = app.actorFor(actorPath.parent)
if (parent.isDefined) {
app.provider.actorOf(Props(creator = actorFactory), parent.get, actorPath.name)
app.provider.actorOf(app, Props(creator = actorFactory), parent.get, actorPath.name)
} else {
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message)
}
@ -180,13 +180,23 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
// }
}
/*
* generate name for temporary actor refs
*/
private val tempNumber = new AtomicLong
def tempName = {
val l = tempNumber.getAndIncrement()
"$_" + Helpers.base64(l)
}
def tempPath = remoteDaemon.path / tempName
// FIXME: handle real remote supervision
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
Props(
context {
case f: Function0[_] try { f() } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
}
// FIXME: handle real remote supervision
@ -195,7 +205,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
Props(
context {
case f: Function0[_] try { sender ! f() } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
}
// FIXME: handle real remote supervision
@ -204,7 +214,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
Props(
context {
case (fun: Function[_, _], param: Any) try { fun.asInstanceOf[Any Unit].apply(param) } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
}
// FIXME: handle real remote supervision
@ -213,7 +223,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
Props(
context {
case (fun: Function[_, _], param: Any) try { sender ! fun.asInstanceOf[Any Any](param) } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
}
def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
@ -230,9 +240,12 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
}
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
val provider = remote.app.asInstanceOf[ActorSystemImpl].provider
lazy val sender: ActorRef =
if (input.hasSender)
remote.app.provider.deserialize(
provider.deserialize(
SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES"))
else
remote.app.deadLetters
@ -282,7 +295,7 @@ trait RemoteMarshallingOps {
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
*/
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
val rep = app.provider.serialize(actor)
val rep = app.asInstanceOf[ActorSystemImpl].provider.serialize(actor)
ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build
}

View file

@ -29,43 +29,51 @@ import akka.event.EventStream
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteActorRefProvider(
val app: ActorSystem,
val AkkaConfig: ActorSystem.AkkaConfig,
val root: ActorPath,
val eventStream: EventStream,
val dispatcher: MessageDispatcher,
val scheduler: Scheduler) extends ActorRefProvider {
val log = Logging(app, this)
val log = Logging(eventStream, this)
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
val local = new LocalActorRefProvider(app, AkkaConfig, root, eventStream, dispatcher, scheduler)
val local = new LocalActorRefProvider(AkkaConfig, root, eventStream, dispatcher, scheduler)
def deathWatch = local.deathWatch
def guardian = local.guardian
def systemGuardian = local.systemGuardian
def nodename = local.nodename
def tempName = local.tempName
val remote = new Remote(app, nodename)
@volatile
var remote: Remote = _
private val actors = new ConcurrentHashMap[String, AnyRef]
private val remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
@volatile
private var remoteDaemonConnectionManager: RemoteConnectionManager = _
def init(app: ActorSystemImpl) {
local.init(app)
remote = new Remote(app, nodename)
remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
}
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime
private[akka] def terminationFuture = local.terminationFuture
private[akka] def deployer: Deployer = local.deployer
def defaultDispatcher = app.dispatcher
def defaultTimeout = app.AkkaConfig.ActorTimeout
def defaultDispatcher = dispatcher
def defaultTimeout = AkkaConfig.ActorTimeout
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(app, props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
if (systemService) local.actorOf(props, supervisor, path, systemService)
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
if (systemService) local.actorOf(app, props, supervisor, path, systemService)
else {
val name = path.name
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
@ -90,7 +98,7 @@ class RemoteActorRefProvider(
if (isReplicaNode) {
// we are on one of the replica node for this remote actor
local.actorOf(props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
local.actorOf(app, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
} else {
// we are on the single "reference" node uses the remote actors on the replica nodes
@ -127,17 +135,17 @@ class RemoteActorRefProvider(
val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a)
val remoteAddress = RemoteAddress(a.hostname, a.port)
conns + (remoteAddress -> RemoteActorRef(remote.server, remoteAddress, path, None))
conns + (remoteAddress -> RemoteActorRef(remote.app.provider, remote.server, remoteAddress, path, None))
}
val connectionManager = new RemoteConnectionManager(app, remote, connections)
connections.keys foreach { useActorOnNode(_, path.toString, props.creator) }
connections.keys foreach { useActorOnNode(app, _, path.toString, props.creator) }
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
}
case deploy local.actorOf(props, supervisor, name, systemService)
case deploy local.actorOf(app, props, supervisor, name, systemService)
}
} catch {
case e: Exception
@ -159,7 +167,7 @@ class RemoteActorRefProvider(
* Copied from LocalActorRefProvider...
*/
// FIXME: implement supervision
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
new RoutedActorRef(app, props, supervisor, name)
}
@ -170,6 +178,7 @@ class RemoteActorRefProvider(
case future: Future[_] Some(future.get.asInstanceOf[ActorRef])
}
// TODO remove me
val optimizeLocal = new AtomicBoolean(true)
def optimizeLocalScoped_?() = optimizeLocal.get
@ -185,19 +194,19 @@ class RemoteActorRefProvider(
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
val remoteAddress = RemoteAddress(actor.hostname, actor.port)
if (optimizeLocalScoped_? && remoteAddress == app.address) {
if (optimizeLocalScoped_? && remoteAddress == root.remoteAddress) {
local.actorFor(ActorPath.split(actor.path))
} else {
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress)
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", root.remoteAddress, actor.path, remoteAddress)
Some(RemoteActorRef(remote.app.provider, remote.server, remoteAddress, root / ActorPath.split(actor.path), None)) //Should it be None here
}
}
/**
* Using (checking out) actor on a specific node.
*/
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) {
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress)
def useActorOnNode(app: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) {
log.debug("[{}] Instantiating Actor [{}] on node [{}]", root, actorPath, remoteAddress)
val actorFactoryBytes =
app.serialization.serialize(actorFactory) match {
@ -260,6 +269,7 @@ class RemoteActorRefProvider(
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[akka] case class RemoteActorRef private[akka] (
provider: ActorRefProvider,
remote: RemoteSupport,
remoteAddress: RemoteAddress,
path: ActorPath,
@ -279,7 +289,7 @@ private[akka] case class RemoteActorRef private[akka] (
def tell(message: Any, sender: ActorRef): Unit = remote.send(message, Option(sender), remoteAddress, this, loader)
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = remote.app.provider.ask(message, this, timeout)
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = provider.ask(message, this, timeout)
def suspend(): Unit = ()
@ -295,7 +305,7 @@ private[akka] case class RemoteActorRef private[akka] (
}
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = remote.app.provider.serialize(this)
private def writeReplace(): AnyRef = provider.serialize(this)
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement

View file

@ -149,5 +149,5 @@ class RemoteConnectionManager(
}
private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) =
RemoteActorRef(remote.server, remoteAddress, actorPath, None)
RemoteActorRef(remote.app.provider, remote.server, remoteAddress, actorPath, None)
}

View file

@ -147,7 +147,8 @@ class ActiveRemoteClient private[akka] (
def sendSecureCookie(connection: ChannelFuture) {
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(remoteSupport.app.address.hostname).setPort(remoteSupport.app.address.port).build)
val addr = remoteSupport.app.root.remoteAddress
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build)
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
}
@ -350,8 +351,8 @@ class ActiveRemoteClientHandler(
*/
class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
val serverSettings = new RemoteServerSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val clientsLock = new ReadWriteGuard
@ -428,7 +429,7 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
def name = currentServer.get match {
case Some(server) server.name
case None "Non-running NettyRemoteServer@" + app.address
case None "Non-running NettyRemoteServer@" + app.root.remoteAddress
}
private val _isRunning = new Switch(false)
@ -459,7 +460,8 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
val log = Logging(remoteSupport.app, this)
import remoteSupport.serverSettings._
import remoteSupport.app.address
val address = remoteSupport.app.root.remoteAddress
val name = "NettyRemoteServer@" + address

View file

@ -5,6 +5,7 @@
package akka.remote
import akka.testkit._
import akka.actor.ActorSystemImpl
abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync {
@ -12,7 +13,7 @@ abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync {
* Helper function for accessing the underlying remoting.
*/
def remote: Remote = {
app.provider match {
system.asInstanceOf[ActorSystemImpl].provider match {
case r: RemoteActorRefProvider r.remote
case _ throw new Exception("Remoting is not enabled")
}

View file

@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec {
class SomeActor extends Actor with Serializable {
def receive = {
case "identify" sender ! app.provider.nodename
case "identify" sender ! system.nodename
}
}
}
@ -47,7 +47,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec {
barrier("start")
val actor = app.actorOf[SomeActor]("service-hello")
val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true)
val result = (actor ? "identify").get

View file

@ -8,7 +8,7 @@ object NewRemoteActorMultiJvmSpec {
class SomeActor extends Actor with Serializable {
def receive = {
case "identify" sender ! app.provider.nodename
case "identify" sender ! system.nodename
}
}
}
@ -46,7 +46,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec {
barrier("start")
val actor = app.actorOf[SomeActor]("service-hello")
val actor = system.actorOf[SomeActor]("service-hello")
val result = (actor ? "identify").get
result must equal("node1")

View file

@ -9,7 +9,7 @@ object RandomRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4
class SomeActor extends Actor with Serializable {
def receive = {
case "hit" sender ! app.provider.nodename
case "hit" sender ! system.nodename
case "end" self.stop()
}
}
@ -70,7 +70,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
remote.start()
barrier("start")
val actor = app.actorOf[SomeActor]("service-hello")
val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true)
val connectionCount = NrOfNodes - 1

View file

@ -9,7 +9,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4
class SomeActor extends Actor with Serializable {
def receive = {
case "hit" sender ! app.provider.nodename
case "hit" sender ! system.nodename
case "end" self.stop()
}
}
@ -70,7 +70,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
remote.start()
barrier("start")
val actor = app.actorOf[SomeActor]("service-hello")
val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true)
val connectionCount = NrOfNodes - 1

View file

@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4
class SomeActor extends Actor with Serializable {
def receive = {
case "hit" sender ! app.provider.nodename
case "hit" sender ! system.nodename
case "end" self.stop()
}
}
@ -70,7 +70,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
remote.start()
barrier("start")
val actor = app.actorOf[SomeActor]("service-hello")
val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true)
actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true)

View file

@ -77,7 +77,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
case Taken(`chopstickToWaitFor`)
println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address)
become(eating)
app.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS)
system.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS)
case Busy(chopstick)
become(thinking)
@ -106,7 +106,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
left ! Put(self)
right ! Put(self)
println("%s puts down his chopsticks and starts to think", name)
app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
}
//All hakkers start in a non-eating state
@ -114,7 +114,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
case Think
println("%s starts to think", name)
become(thinking)
app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
}
}

View file

@ -32,7 +32,7 @@ case class TakenBy(hakker: ActorRef)
class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] {
// A chopstick begins its existence as available and taken by no one
startWith(Available, TakenBy(app.deadLetters))
startWith(Available, TakenBy(system.deadLetters))
// When a chopstick is available, it can be taken by a some hakker
when(Available) {
@ -47,7 +47,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] {
case Event(Take, currentState)
stay replying Busy(self)
case Event(Put, TakenBy(hakker)) if sender == hakker
goto(Available) using TakenBy(app.deadLetters)
goto(Available) using TakenBy(system.deadLetters)
}
// Initialze the chopstick

View file

@ -9,7 +9,7 @@ public class EitherOrElseExample {
System.out.println("EitherOrElse example");
System.out.println();
ActorSystem application = new ActorSystem("UntypedTransactorExample");
ActorSystem application = ActorSystem.create("UntypedTransactorExample");
final Ref<Integer> left = new Ref<Integer>(100);
final Ref<Integer> right = new Ref<Integer>(100);

View file

@ -10,7 +10,7 @@ public class RetryExample {
System.out.println("Retry example");
System.out.println();
ActorSystem application = new ActorSystem("RetryExample");
ActorSystem application = ActorSystem.create("RetryExample");
final Ref<Double> account1 = new Ref<Double>(100.0);
final Ref<Double> account2 = new Ref<Double>(100.0);

View file

@ -12,7 +12,7 @@ public class UntypedCoordinatedExample {
System.out.println("Untyped transactor example");
System.out.println();
ActorSystem application = new ActorSystem("UntypedCoordinatedExample");
ActorSystem application = ActorSystem.create("UntypedCoordinatedExample");
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));

View file

@ -11,7 +11,7 @@ public class UntypedTransactorExample {
System.out.println("Untyped transactor example");
System.out.println();
ActorSystem application = new ActorSystem("UntypedTransactorExample");
ActorSystem application = ActorSystem.create("UntypedTransactorExample");
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class));
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class));

View file

@ -30,7 +30,7 @@ import scala.collection.JavaConverters;
import scala.collection.Seq;
public class UntypedCoordinatedIncrementTest {
ActorSystem application = new ActorSystem("UntypedCoordinatedIncrementTest");
ActorSystem application = ActorSystem.create("UntypedCoordinatedIncrementTest");
List<ActorRef> counters;
ActorRef failer;

View file

@ -27,7 +27,7 @@ import scala.collection.JavaConverters;
import scala.collection.Seq;
public class UntypedTransactorTest {
ActorSystem application = new ActorSystem("UntypedTransactorTest");
ActorSystem application = ActorSystem.create("UntypedTransactorTest");
List<ActorRef> counters;
ActorRef failer;

View file

@ -16,7 +16,7 @@ class ConfigSpec extends WordSpec with MustMatchers {
"The default configuration file (i.e. akka-reference.conf)" should {
"contain all configuration properties for akka-stm that are used in code with their correct defaults" in {
val config = ActorSystem("ConfigSpec").config
val config = ActorSystem("ConfigSpec").AkkaConfig.config
import config._

View file

@ -61,9 +61,9 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll {
val numCounters = 4
def actorOfs = {
def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i)))
def createCounter(i: Int) = system.actorOf(Props(new Counter("counter" + i)))
val counters = (1 to numCounters) map createCounter
val failer = app.actorOf(Props(new Failer))
val failer = system.actorOf(Props(new Failer))
(counters, failer)
}

View file

@ -104,9 +104,9 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
val numCounters = 2
def actorOfs = {
def createCounter(i: Int) = app.actorOf(Props(new FickleCounter("counter" + i)))
def createCounter(i: Int) = system.actorOf(Props(new FickleCounter("counter" + i)))
val counters = (1 to numCounters) map createCounter
val coordinator = app.actorOf(Props(new Coordinator("coordinator")))
val coordinator = system.actorOf(Props(new Coordinator("coordinator")))
(counters, coordinator)
}
@ -116,7 +116,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
EventFilter[ExpectedFailureException](),
EventFilter[CoordinatedTransactionException](),
EventFilter[ActorTimeoutException]())
app.eventStream.publish(Mute(ignoreExceptions))
system.eventStream.publish(Mute(ignoreExceptions))
val (counters, coordinator) = actorOfs
val latch = new CountDownLatch(1)
coordinator ! FriendlyIncrement(counters, latch)

View file

@ -84,9 +84,9 @@ class TransactorSpec extends AkkaSpec {
val numCounters = 3
def createTransactors = {
def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i)))
def createCounter(i: Int) = system.actorOf(Props(new Counter("counter" + i)))
val counters = (1 to numCounters) map createCounter
val failer = app.actorOf(Props(new Failer))
val failer = system.actorOf(Props(new Failer))
(counters, failer)
}
@ -124,7 +124,7 @@ class TransactorSpec extends AkkaSpec {
"Transactor" should {
"be usable without overriding normally" in {
val transactor = app.actorOf(Props(new Setter))
val transactor = system.actorOf(Props(new Setter))
val ref = Ref(0)
val latch = TestLatch(1)
transactor ! Set(ref, 5, latch)

View file

@ -22,7 +22,7 @@ import akka.event.EventStream
* @since 1.1
*/
class TestActorRef[T <: Actor](
_app: ActorSystem,
_app: ActorSystemImpl,
_deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
@ -63,10 +63,12 @@ object TestActorRef {
def apply[T <: Actor](props: Props)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, randomName)
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, name: String)(implicit app: ActorSystem): TestActorRef[T] =
apply[T](props, app.asInstanceOf[ActorSystemImpl].guardian, name)
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = {
new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name)
val impl = app.asInstanceOf[ActorSystemImpl]
new TestActorRef(impl, impl.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name)
}
def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName)

View file

@ -37,7 +37,7 @@ import akka.event.EventStream
* @since 1.2
*/
class TestFSMRef[S, D, T <: Actor](
app: ActorSystem,
app: ActorSystemImpl,
_deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
@ -89,9 +89,13 @@ class TestFSMRef[S, D, T <: Actor](
object TestFSMRef {
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () factory), app.guardian, TestActorRef.randomName)
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = {
val impl = app.asInstanceOf[ActorSystemImpl]
new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () factory), impl.guardian, TestActorRef.randomName)
}
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, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () factory), app.guardian, name)
def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = {
val impl = app.asInstanceOf[ActorSystemImpl]
new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () factory), impl.guardian, name)
}
}

View file

@ -80,7 +80,7 @@ class TestKit(_app: ActorSystem) {
import TestActor.{ Message, RealMessage, NullMessage }
implicit val app = _app
implicit val system = _app
private val queue = new LinkedBlockingDeque[Message]()
private[akka] var lastMessage: Message = NullMessage
@ -91,9 +91,12 @@ class TestKit(_app: ActorSystem) {
* ActorRef of the test actor. Access is provided to enable e.g.
* registration as message target.
*/
val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue))
.copy(dispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler)),
val testActor: ActorRef = {
val impl = system.asInstanceOf[ActorSystemImpl]
impl.systemActorOf(Props(new TestActor(queue))
.copy(dispatcher = new CallingThreadDispatcher(impl.deadLetterMailbox, impl.eventStream, impl.scheduler)),
"testActor" + TestKit.testActorId.incrementAndGet)
}
private var end: Duration = Duration.Undefined
@ -124,7 +127,7 @@ class TestKit(_app: ActorSystem) {
* block or missing that it returns the properly dilated default for this
* case from AkkaConfig (key "akka.test.single-expect-default").
*/
def remaining: Duration = if (end == Duration.Undefined) app.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now
def remaining: Duration = if (end == Duration.Undefined) system.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now
/**
* Query queue status.

View file

@ -6,7 +6,7 @@ package akka.testkit
import akka.config.Configuration
import org.scalatest.{ WordSpec, BeforeAndAfterAll }
import org.scalatest.matchers.MustMatchers
import akka.actor.ActorSystem
import akka.actor.{ ActorSystem, ActorSystemImpl }
import akka.actor.{ Actor, ActorRef, Props }
import akka.dispatch.MessageDispatcher
import akka.event.{ Logging, LoggingAdapter }
@ -16,16 +16,16 @@ import akka.dispatch.FutureTimeoutException
abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll {
val log: LoggingAdapter = Logging(app.eventStream, this)
val log: LoggingAdapter = Logging(system, this)
final override def beforeAll {
atStartup()
}
final override def afterAll {
app.stop()
try app.terminationFuture.await(5 seconds) catch {
case _: FutureTimeoutException app.log.warning("failed to stop within 5 seconds")
system.stop()
try system.asInstanceOf[ActorSystemImpl].terminationFuture.await(5 seconds) catch {
case _: FutureTimeoutException system.log.warning("failed to stop within 5 seconds")
}
atTermination()
}
@ -34,9 +34,9 @@ abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
protected def atTermination() {}
def this(config: Configuration) = this(new ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config))
def this(config: Configuration) = this(ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config))
def actorOf(props: Props): ActorRef = app.actorOf(props)
def actorOf(props: Props): ActorRef = system.actorOf(props)
def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz))

View file

@ -170,8 +170,8 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
counter = 2
val boss = TestActorRef(Props(new TActor {
val ref = new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(new TActor {
val impl = system.asInstanceOf[ActorSystemImpl]
val ref = new TestActorRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(new TActor {
def receiveT = { case _ }
override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 }
override def postRestart(reason: Throwable) { counter -= 1 }

View file

@ -15,7 +15,7 @@ class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0)
val now = System.nanoTime
intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) }
val diff = System.nanoTime - now
val target = (1000000000l * app.AkkaConfig.TestTimeFactor).toLong
val target = (1000000000l * system.AkkaConfig.TestTimeFactor).toLong
diff must be > (target - 300000000l)
diff must be < (target + 1000000000l)
}