Adding DispatcherPrerequisites to hold the common dependencies that a dispatcher needs to be created

This commit is contained in:
Viktor Klang 2011-11-17 16:09:18 +01:00
commit 80d766b07b
139 changed files with 1126 additions and 948 deletions

View file

@ -9,15 +9,15 @@ import static org.junit.Assert.*;
public class JavaAPI {
private ActorSystem app = new ActorSystem();
private ActorSystem system = ActorSystem.create();
@Test void mustBeAbleToCreateActorRefFromClass() {
ActorRef ref = app.actorOf(JavaAPITestActor.class);
ActorRef ref = system.actorOf(JavaAPITestActor.class);
assertNotNull(ref);
}
@Test void mustBeAbleToCreateActorRefFromFactory() {
ActorRef ref = app.actorOf(new Props().withCreator(new Creator<Actor>() {
ActorRef ref = system.actorOf(new Props().withCreator(new Creator<Actor>() {
public Actor create() {
return new JavaAPITestActor();
}
@ -26,7 +26,7 @@ public class JavaAPI {
}
@Test void mustAcceptSingleArgTell() {
ActorRef ref = app.actorOf(JavaAPITestActor.class);
ActorRef ref = system.actorOf(JavaAPITestActor.class);
ref.tell("hallo");
ref.tell("hallo", ref);
}

View file

@ -19,9 +19,9 @@ import scala.Right;
public class JavaFutureTests {
private final ActorSystem app = new ActorSystem();
private final Timeout t = app.AkkaConfig().ActorTimeout();
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);
private final ActorSystem system = ActorSystem.create();
private final Timeout t = system.settings().ActorTimeout();
private final FutureFactory ff = new FutureFactory(system.dispatcher(), t);
@Test public void mustBeAbleToMapAFuture() {
Future<String> f1 = ff.future(new Callable<String>() {
@ -41,7 +41,7 @@ public class JavaFutureTests {
@Test public void mustBeAbleToExecuteAnOnResultCallback() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
Future<String> f = cf;
f.onResult(new Procedure<String>() {
public void apply(String result) {
@ -57,7 +57,7 @@ public class JavaFutureTests {
@Test public void mustBeAbleToExecuteAnOnExceptionCallback() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
Future<String> f = cf;
f.onException(new Procedure<Throwable>() {
public void apply(Throwable t) {
@ -74,7 +74,7 @@ public class JavaFutureTests {
@Test public void mustBeAbleToExecuteAnOnTimeoutCallback() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
Future<String> f = cf;
f.onTimeout(new Procedure<Future<String>>() {
public void apply(Future<String> future) {
@ -88,7 +88,7 @@ public class JavaFutureTests {
@Test public void mustBeAbleToExecuteAnOnCompleteCallback() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
Future<String> f = cf;
f.onComplete(new Procedure<Future<String>>() {
public void apply(akka.dispatch.Future<String> future) {
@ -103,7 +103,7 @@ public class JavaFutureTests {
@Test public void mustBeAbleToForeachAFuture() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
Future<String> f = cf;
f.foreach(new Procedure<String>() {
public void apply(String future) {
@ -118,13 +118,13 @@ public class JavaFutureTests {
@Test public void mustBeAbleToFlatMapAFuture() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
cf.completeWithResult("1000");
Future<String> f = cf;
Future<Integer> r = f.flatMap(new Function<String, Future<Integer>>() {
public Future<Integer> apply(String r) {
latch.countDown();
Promise<Integer> cf = new akka.dispatch.DefaultPromise<Integer>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<Integer> cf = new akka.dispatch.DefaultPromise<Integer>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
cf.completeWithResult(Integer.parseInt(r));
return cf;
}
@ -137,7 +137,7 @@ public class JavaFutureTests {
@Test public void mustBeAbleToFilterAFuture() throws Throwable {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
Future<String> f = cf;
Future<String> r = f.filter(new Function<String, Boolean>() {
public Boolean apply(String r) {

View file

@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec {
out.flush
out.close
Serialization.app.withValue(app) {
Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val readA = in.readObject
@ -257,7 +257,7 @@ class ActorRefSpec extends AkkaSpec {
}
}
"throw an exception on deserialize if no app in scope" in {
"throw an exception on deserialize if no system in scope" in {
val a = actorOf[InnerActor]
import java.io._
@ -275,7 +275,7 @@ class ActorRefSpec extends AkkaSpec {
(intercept[java.lang.IllegalStateException] {
in.readObject
}).getMessage must be === "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }"
" Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }"
}
"must throw exception on deserialize if not present in actor hierarchy (and remoting is not enabled)" in {
@ -284,14 +284,15 @@ 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 addr = system.rootPath.remoteAddress
val serialized = SerializedActorRef(addr.hostname, addr.port, "/this/path/does/not/exist")
out.writeObject(serialized)
out.flush
out.close
Serialization.app.withValue(app) {
Serialization.system.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.settings.ActorTimeout.duration
val testTimeout = if (system.settings.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.settings.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

@ -17,7 +17,7 @@ import akka.config.Configuration
object FSMActorSpec {
class Latches(implicit app: ActorSystem) {
class Latches(implicit system: ActorSystem) {
val unlockedLatch = TestLatch()
val lockedLatch = TestLatch()
val unhandledLatch = TestLatch()
@ -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

@ -24,7 +24,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
"A Finite State Machine" must {
"receive StateTimeout" in {
"receive StateTimeout" taggedAs TimingTest in {
within(1 second) {
within(500 millis, 1 second) {
fsm ! TestStateTimeout
@ -34,7 +34,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
}
}
"cancel a StateTimeout" in {
"cancel a StateTimeout" taggedAs TimingTest in {
within(1 second) {
fsm ! TestStateTimeout
fsm ! Cancel
@ -44,7 +44,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
}
}
"allow StateTimeout override" in {
"allow StateTimeout override" taggedAs TimingTest in {
within(500 millis) {
fsm ! TestStateTimeoutOverride
expectNoMsg
@ -56,8 +56,8 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
}
}
"receive single-shot timer" in {
within(1.5 seconds) {
"receive single-shot timer" taggedAs TimingTest in {
within(2 seconds) {
within(500 millis, 1 second) {
fsm ! TestSingleTimer
expectMsg(Tick)
@ -67,7 +67,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
}
}
"correctly cancel a named timer" in {
"correctly cancel a named timer" taggedAs TimingTest in {
fsm ! TestCancelTimer
within(500 millis) {
fsm ! Tick
@ -80,7 +80,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
expectMsg(1 second, Transition(fsm, TestCancelTimer, Initial))
}
"not get confused between named and state timers" in {
"not get confused between named and state timers" taggedAs TimingTest in {
fsm ! TestCancelStateTimerInNamedTimerMessage
fsm ! Tick
expectMsg(500 millis, Tick)
@ -94,7 +94,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
}
}
"receive and cancel a repeated timer" in {
"receive and cancel a repeated timer" taggedAs TimingTest in {
fsm ! TestRepeatedTimer
val seq = receiveWhile(2 seconds) {
case Tick Tick
@ -105,7 +105,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
}
}
"notify unhandled messages" in {
"notify unhandled messages" taggedAs TimingTest in {
filterEvents(EventFilter.warning("unhandled event Tick in state TestUnhandled", source = fsm, occurrences = 1),
EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm, occurrences = 1)) {
fsm ! TestUnhandled

View file

@ -12,12 +12,12 @@ import akka.util.Duration
object ForwardActorSpec {
val ExpectedMessage = "FOO"
def createForwardingChain(app: ActorSystem): ActorRef = {
val replier = app.actorOf(new Actor {
def createForwardingChain(system: ActorSystem): ActorRef = {
val replier = system.actorOf(new Actor {
def receive = { case x sender ! x }
})
def mkforwarder(forwardTo: ActorRef) = app.actorOf(
def mkforwarder(forwardTo: ActorRef) = system.actorOf(
new Actor {
def receive = { case x forwardTo forward x }
})
@ -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) {
@ -150,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
expectNoMsg(Duration.Zero)
assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)")
val actor = new TestActorRef[TestLogActor](app, Props[TestLogActor], supervisor, "none")
val actor = TestActorRef[TestLogActor](Props[TestLogActor], supervisor, "none")
val set = receiveWhile(messages = 2) {
case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" 1

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)
@ -68,10 +68,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
// val actor = actorOf(new Actor {
// def receive = { case Ping ticks.countDown }
// })
// val numActors = app.registry.local.actors.length
// val numActors = system.registry.local.actors.length
// (1 to 1000).foreach(_ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS)))
// assert(ticks.await(10, TimeUnit.SECONDS))
// assert(app.registry.local.actors.length === numActors)
// assert(system.registry.local.actors.length === numActors)
// }
/**
@ -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, 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.system.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, 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.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method)

View file

@ -141,13 +141,13 @@ object ActorModelSpec {
}
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
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)(stops == dispatcher.stops.get)
} catch {
case e
app.eventStream.publish(Error(e, dispatcher, "actual: stops=" + dispatcher.stops.get +
system.eventStream.publish(Error(e, dispatcher, "actual: stops=" + dispatcher.stops.get +
" required: stops=" + stops))
throw e
}
@ -173,7 +173,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,
@ -191,7 +191,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 {
@ -204,7 +204,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
@ -227,7 +227,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
@ -310,7 +310,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"))
}
}
}
@ -407,10 +407,10 @@ class DispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, "foo", app.AkkaConfig.DispatcherThroughput,
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor]
new Dispatcher(system.dispatcherFactory.prerequisites, "foo", system.settings.DispatcherThroughput,
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Dispatcher"
@ -444,14 +444,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, "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)).build.asInstanceOf[MessageDispatcherInterceptor]
new BalancingDispatcher(system.dispatcherFactory.prerequisites, "foo", 1, // TODO check why 1 here? (came from old test)
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig()).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.settings.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

@ -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.settings.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)
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)
bus.startDefaultLoggers(app, 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)
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

@ -12,8 +12,9 @@ import java.util.concurrent.ThreadPoolExecutor.AbortPolicy
class TellThroughputPerformanceSpec extends PerformanceSpec {
import TellThroughputPerformanceSpec._
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config new Dispatcher(app, name, 5,
0, UnboundedMailbox(), config, 60000), ThreadPoolConfig(app))
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
0, UnboundedMailbox(), config, 60000), ThreadPoolConfig())
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
@ -71,9 +72,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

@ -34,7 +34,7 @@ trait TradingSystem {
case class MatchingEngineInfo(primary: ME, standby: Option[ME], orderbooks: List[Orderbook])
}
class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
class AkkaTradingSystem(val system: ActorSystem) extends TradingSystem {
type ME = ActorRef
type OR = ActorRef
@ -70,8 +70,8 @@ class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) =
meDispatcher match {
case Some(d) app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d))
case _ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)))
case Some(d) system.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d))
case _ system.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)))
}
override def createOrderReceivers: List[ActorRef] = {
@ -91,8 +91,8 @@ class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
}
def createOrderReceiver() = orDispatcher match {
case Some(d) app.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d))
case _ app.actorOf(Props(new AkkaOrderReceiver()))
case Some(d) system.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d))
case _ system.actorOf(Props(new AkkaOrderReceiver()))
}
override def start() {

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

@ -10,12 +10,12 @@ import akka.event.Logging
import scala.collection.immutable.TreeMap
class Report(
app: ActorSystem,
system: ActorSystem,
resultRepository: BenchResultRepository,
compareResultWith: Option[String] = None) {
private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean
val log = Logging(app, this)
val log = Logging(system, this)
val dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
@ -221,11 +221,11 @@ class Report(
sb.append("Args:\n ").append(args)
sb.append("\n")
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
sb.append("Akka version: ").append(system.settings.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 system.settings.config.keys) {
sb.append("\n ").append(key).append("=").append(system.settings.config(key))
}
sb.toString

View file

@ -29,7 +29,7 @@ class TypedActorPoolSpec extends AkkaSpec {
import ActorPoolSpec._
"Actor Pool (2)" must {
"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
@ -38,7 +38,7 @@ class TypedActorPoolSpec 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))
@ -47,7 +47,7 @@ class TypedActorPoolSpec extends AkkaSpec {
for ((i, r) results)
r.get must equal(i * i)
app.typedActor.stop(pool)
system.typedActor.stop(pool)
}
}
}

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.system.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, "test") with MessageDispatcherInterceptor
def newInterceptedDispatcher = new CallingThreadDispatcher(system.dispatcherFactory.prerequisites, "test") with MessageDispatcherInterceptor
def dispatcherType = "Calling Thread Dispatcher"
}

View file

@ -150,11 +150,11 @@ object Timeout {
implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
implicit def defaultTimeout(implicit app: ActorSystem) = app.AkkaConfig.ActorTimeout
implicit def defaultTimeout(implicit system: ActorSystem) = system.settings.ActorTimeout
}
trait ActorLogging { this: Actor
val log = akka.event.Logging(app.eventStream, context.self)
val log = akka.event.Logging(system.eventStream, context.self)
}
object Actor {
@ -164,17 +164,17 @@ object Actor {
/**
* This decorator adds invocation logging to a Receive function.
*/
class LoggingReceive(source: AnyRef, r: Receive)(implicit app: ActorSystem) extends Receive {
class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive {
def isDefinedAt(o: Any) = {
val handled = r.isDefinedAt(o)
app.eventStream.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
system.eventStream.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
handled
}
def apply(o: Any): Unit = r(o)
}
object LoggingReceive {
def apply(source: AnyRef, r: Receive)(implicit app: ActorSystem): Receive = r match {
def apply(source: AnyRef, r: Receive)(implicit system: ActorSystem): Receive = r match {
case _: LoggingReceive r
case _ new LoggingReceive(source, r)
}
@ -229,12 +229,12 @@ trait Actor {
c
}
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.settings.ActorTimeout
/**
* Wrap a Receive partial function in a logging enclosure, which sends a
@ -250,7 +250,7 @@ trait Actor {
* This method does NOT modify the given Receive unless
* akka.actor.debug.receive is set within akka.conf.
*/
def loggable(self: AnyRef)(r: Receive): Receive = if (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.settings.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
/**
* The 'self' field holds the ActorRef for this actor.

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 system: ActorSystemImpl,
val self: ActorRef with ScalaActorRef,
val props: Props,
val parent: ActorRef,
@ -72,11 +72,13 @@ private[akka] class ActorCell(
import ActorCell._
def systemImpl = system
protected final def guardian = self
protected def typedActor = app.typedActor
protected def typedActor = system.typedActor
final def provider = app.provider
final def provider = system.provider
var futureTimeout: Option[Cancellable] = None
@ -91,7 +93,7 @@ private[akka] class ActorCell(
var stopping = false
@inline
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
final def isShutdown: Boolean = mailbox.isClosed
@ -139,12 +141,12 @@ private[akka] class ActorCell(
}
final def tell(message: Any, sender: ActorRef): Unit =
dispatcher.dispatch(this, Envelope(message, if (sender eq null) app.deadLetters else sender))
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender))
final def sender: ActorRef = currentMessage match {
case null app.deadLetters
case null system.deadLetters
case msg if msg.sender ne null msg.sender
case _ app.deadLetters
case _ system.deadLetters
}
//This method is in charge of setting up the contextStack and create a new instance of the Actor
@ -172,11 +174,11 @@ private[akka] class ActorCell(
actor = created
created.preStart()
checkReceiveTimeout
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")"))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "started (" + actor + ")"))
} catch {
case e
try {
app.eventStream.publish(Error(e, self, "error while creating actor"))
system.eventStream.publish(Error(e, self, "error while creating actor"))
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
} finally {
@ -186,7 +188,7 @@ private[akka] class ActorCell(
def recreate(cause: Throwable): Unit = try {
val failedActor = actor
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting"))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "restarting"))
val freshActor = newActor()
if (failedActor ne null) {
val c = currentMessage //One read only plz
@ -200,14 +202,14 @@ private[akka] class ActorCell(
}
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
freshActor.postRestart(cause)
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted"))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "restarted"))
dispatcher.resume(this) //FIXME should this be moved down?
props.faultHandler.handleSupervisorRestarted(cause, self, children)
} catch {
case e try {
app.eventStream.publish(Error(e, self, "error while creating actor"))
system.eventStream.publish(Error(e, self, "error while creating actor"))
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
} finally {
@ -226,7 +228,7 @@ private[akka] class ActorCell(
val c = children
if (c.isEmpty) doTerminate()
else {
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping"))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopping"))
for (child c) child.stop()
stopping = true
}
@ -237,8 +239,8 @@ private[akka] class ActorCell(
if (!stats.contains(child)) {
childrenRefs = childrenRefs.updated(child.name, child)
childrenStats = childrenStats.updated(child, ChildRestartStats())
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child))
} else app.eventStream.publish(Warning(self, "Already supervising " + child))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now supervising " + child))
} else system.eventStream.publish(Warning(self, "Already supervising " + child))
}
try {
@ -252,11 +254,11 @@ private[akka] class ActorCell(
case Create() create()
case Recreate(cause) recreate(cause)
case Link(subject)
app.deathWatch.subscribe(self, subject)
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject))
system.deathWatch.subscribe(self, subject)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now monitoring " + subject))
case Unlink(subject)
app.deathWatch.unsubscribe(self, subject)
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject))
system.deathWatch.unsubscribe(self, subject)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped monitoring " + subject))
case Suspend() suspend()
case Resume() resume()
case Terminate() terminate()
@ -265,7 +267,7 @@ private[akka] class ActorCell(
}
} catch {
case e //Should we really catch everything here?
app.eventStream.publish(Error(e, self, "error while processing " + message))
system.eventStream.publish(Error(e, self, "error while processing " + message))
//TODO FIXME How should problems here be handled?
throw e
}
@ -284,7 +286,7 @@ private[akka] class ActorCell(
case msg
if (stopping) {
// receiving Terminated in response to stopping children is too common to generate noise
if (!msg.isInstanceOf[Terminated]) app.deadLetterMailbox.enqueue(self, messageHandle)
if (!msg.isInstanceOf[Terminated]) system.deadLetterMailbox.enqueue(self, messageHandle)
} else {
actor(msg)
}
@ -292,7 +294,7 @@ private[akka] class ActorCell(
currentMessage = null // reset current message after successful invocation
} catch {
case e
app.eventStream.publish(Error(e, self, e.getMessage))
system.eventStream.publish(Error(e, self, e.getMessage))
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
@ -312,7 +314,7 @@ private[akka] class ActorCell(
}
} catch {
case e
app.eventStream.publish(Error(e, self, e.getMessage))
system.eventStream.publish(Error(e, self, e.getMessage))
throw e
}
}
@ -330,11 +332,11 @@ private[akka] class ActorCell(
}
def autoReceiveMessage(msg: Envelope) {
if (app.AkkaConfig.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg))
if (system.settings.DebugAutoReceive) system.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg))
if (stopping) msg.message match {
case ChildTerminated handleChildTerminated(sender)
case _ app.deadLetterMailbox.enqueue(self, msg)
case _ system.deadLetterMailbox.enqueue(self, msg)
}
else msg.message match {
case HotSwap(code, discardOld) become(code(self), discardOld)
@ -347,8 +349,8 @@ private[akka] class ActorCell(
}
private def doTerminate() {
if (!app.provider.evict(self.path.toString))
app.eventStream.publish(Warning(self, "evict of " + self.path.toString + " failed"))
if (!system.provider.evict(self.path.toString))
system.eventStream.publish(Warning(self, "evict of " + self.path.toString + " failed"))
dispatcher.detach(this)
@ -358,8 +360,8 @@ private[akka] class ActorCell(
} finally {
try {
parent.tell(ChildTerminated, self)
app.deathWatch.publish(Terminated(self))
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped"))
system.deathWatch.publish(Terminated(self))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped"))
} finally {
currentMessage = null
clearActorFields()
@ -369,7 +371,7 @@ private[akka] class ActorCell(
final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenStats.get(child) match {
case Some(stats) if (!props.faultHandler.handleFailure(child, cause, stats, childrenStats)) throw cause
case None app.eventStream.publish(Warning(self, "dropping Failed(" + cause + ") from unknown child"))
case None system.eventStream.publish(Warning(self, "dropping Failed(" + cause + ") from unknown child"))
}
final def handleChildTerminated(child: ActorRef): Unit = {
@ -387,7 +389,7 @@ private[akka] class ActorCell(
val recvtimeout = receiveTimeout
if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) {
//Only reschedule if desired and there are currently no more messages to be processed
futureTimeout = Some(app.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS))
futureTimeout = Some(system.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS))
}
}

View file

@ -4,6 +4,8 @@
package akka.actor
import akka.remote.RemoteAddress
object ActorPath {
final val separator = "/"
@ -12,14 +14,14 @@ object ActorPath {
/**
* Create an actor path from a string.
*/
def apply(app: ActorSystem, path: String): ActorPath =
apply(app, split(path))
def apply(system: ActorSystem, path: String): ActorPath =
apply(system, split(path))
/**
* Create an actor path from an iterable.
*/
def apply(app: ActorSystem, path: Iterable[String]): ActorPath =
path.foldLeft(app.root)(_ / _)
def apply(system: ActorSystem, path: Iterable[String]): ActorPath =
path.foldLeft(system.rootPath)(_ / _)
/**
* Split a string path into an iterable.
@ -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

@ -12,6 +12,8 @@ import akka.serialization.Serialization
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.
@ -160,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,
system: ActorSystemImpl,
_props: Props,
_supervisor: ActorRef,
val path: ActorPath,
@ -171,10 +173,19 @@ 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
* means that messages may be processed theoretically before the constructor
* ends. The JMM guarantees visibility for final fields only after the end
* of the constructor, so publish the actorCell safely by making it a
* @volatile var which is NOT TO BE WRITTEN TO. The alternative would be to
* move start() outside of the constructor, which would basically require
* us to use purely factory methods for creating LocalActorRefs.
*/
@volatile
private var actorCell = new ActorCell(app, this, _props, _supervisor, _receiveTimeout, _hotswap)
private var actorCell = new ActorCell(system, this, _props, _supervisor, _receiveTimeout, _hotswap)
actorCell.start()
/**
@ -295,17 +306,17 @@ trait ScalaActorRef { ref: ActorRef ⇒
*/
case class SerializedActorRef(hostname: String, port: Int, path: String) {
import akka.serialization.Serialization.app
import akka.serialization.Serialization.system
def this(remoteAddress: RemoteAddress, path: String) = this(remoteAddress.hostname, remoteAddress.port, path)
def this(remoteAddress: InetSocketAddress, path: String) = this(remoteAddress.getAddress.getHostAddress, remoteAddress.getPort, path) //TODO FIXME REMOVE
@throws(classOf[java.io.ObjectStreamException])
def readResolve(): AnyRef = {
if (app.value eq null) throw new IllegalStateException(
if (system.value eq null) throw new IllegalStateException(
"Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }")
app.value.provider.deserialize(this) match {
" Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }")
system.value.provider.deserialize(this) match {
case Some(actor) actor
case None throw new IllegalStateException("Could not deserialize ActorRef")
}
@ -344,31 +355,35 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
object DeadLetterActorRef {
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
@throws(classOf[java.io.ObjectStreamException])
private def readResolve(): AnyRef = Serialization.app.value.deadLetters
private def readResolve(): AnyRef = Serialization.system.value.deadLetters
}
val serialized = new SerializedDeadLetterActorRef
}
class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) extends MinimalActorRef {
@volatile
var brokenPromise: Future[Any] = _
private[akka] def init(dispatcher: MessageDispatcher) {
brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher)
}
override val name: String = "dead-letter"
// FIXME (actor path): put this under the sys guardian supervisor
val path: ActorPath = app.root / "sys" / name
def address: String = app.address + path.toString
def address: String = path.toString
override def isShutdown(): Boolean = true
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
case d: DeadLetter app.eventStream.publish(d)
case _ app.eventStream.publish(DeadLetter(message, sender, this))
case d: DeadLetter eventStream.publish(d)
case _ eventStream.publish(DeadLetter(message, sender, this))
}
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
app.eventStream.publish(DeadLetter(message, app.provider.dummyAskSender, this))
eventStream.publish(DeadLetter(message, this, this))
// leave this in: guard with good visibility against really stupid/weird errors
assert(brokenPromise != null)
brokenPromise
}
@ -376,16 +391,15 @@ class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
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
}
@ -411,5 +425,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

@ -5,41 +5,59 @@
package akka.actor
import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.{ TimeUnit, Executors }
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit }
import scala.annotation.tailrec
import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer }
import akka.actor.Timeout.intToTimeout
import akka.config.ConfigurationException
import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise }
import akka.event.{ Logging, DeathWatch, ActorClassification }
import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise, Dispatcher, Mailbox, Envelope }
import akka.event.{ Logging, DeathWatch, ActorClassification, EventStream }
import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter }
import akka.util.Helpers
import akka.AkkaException
import com.eaio.uuid.UUID
/**
* Interface for all ActorRef providers to implement.
*/
trait ActorRefProvider {
def actorOf(props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(props, supervisor, name, false)
def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(system, props, supervisor, name, false)
def actorFor(path: Iterable[String]): Option[ActorRef]
/**
* What deployer will be used to resolve deployment configuration?
*/
def guardian: ActorRef
def systemGuardian: ActorRef
def deathWatch: DeathWatch
// FIXME: remove/replace
def nodename: String
def settings: ActorSystem.Settings
def init(system: ActorSystemImpl)
private[akka] def deployer: Deployer
private[akka] def scheduler: Scheduler
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
/**
* Create an Actor with the given name below the given supervisor.
*/
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
/**
* Create an Actor with the given full path below the given supervisor.
*
* FIXME: Remove! this is dangerous!
*/
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
/**
* Remove this path from the lookup map.
*/
private[akka] def evict(path: String): Boolean
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef]
@ -48,15 +66,16 @@ trait ActorRefProvider {
private[akka] def createDeathWatch(): DeathWatch
/**
* Create AskActorRef to hook up message send to recipient with Future receiver.
*/
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any]
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef
/**
* This Future is completed upon termination of this ActorRefProvider, which
* is usually initiated by stopping the guardian via ActorSystem.stop().
*/
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
private[akka] def dummyAskSender: ActorRef
private[akka] def tempPath: String
}
/**
@ -64,9 +83,11 @@ trait ActorRefProvider {
*/
trait ActorRefFactory {
def provider: ActorRefProvider
protected def systemImpl: ActorSystemImpl
def dispatcher: MessageDispatcher
protected def provider: ActorRefProvider
protected def dispatcher: MessageDispatcher
/**
* Father of all children created by this interface.
@ -80,7 +101,7 @@ trait ActorRefFactory {
Helpers.base64(l)
}
def actorOf(props: Props): ActorRef = provider.actorOf(props, guardian, randomName, false)
def actorOf(props: Props): ActorRef = provider.actorOf(systemImpl, props, guardian, randomName, false)
/*
* TODO this will have to go at some point, because creating two actors with
@ -90,7 +111,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(systemImpl, props, guardian, name, false)
}
def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]))
@ -104,6 +125,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)
@ -114,20 +137,37 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
/**
* Local ActorRef provider.
*/
class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
class LocalActorRefProvider(
val settings: ActorSystem.Settings,
val rootPath: ActorPath,
val eventStream: EventStream,
val dispatcher: MessageDispatcher,
val scheduler: Scheduler) extends ActorRefProvider {
val log = Logging(app.eventStream, this)
val log = Logging(eventStream, this)
private[akka] val deployer: Deployer = new Deployer(app)
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher)
private[akka] val scheduler: Scheduler = { //TODO FIXME Make this configurable
val s = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
terminationFuture.onComplete(_ s.stop())
s
// FIXME remove/replave (clustering shall not leak into akka-actor)
val nodename: String = System.getProperty("akka.cluster.nodename") match {
case null | "" new UUID().toString
case value value
}
private[akka] val deployer: Deployer = new Deployer(settings, eventStream, nodename)
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher)
/*
* generate name for temporary actor refs
*/
private val tempNumber = new AtomicLong
def tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement())
private val tempNode = rootPath / "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)
private val actors = new ConcurrentHashMap[String, AnyRef]
/**
* Top-level anchor for the supervision hierarchy of this actor system. Will
* receive only Supervise/ChildTerminated system messages or Failure message.
@ -136,12 +176,12 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
@volatile
var stopped = false
override val name = app.name + "-bubble-walker"
override val name = "bubble-walker"
// FIXME (actor path): move the root path to the new root guardian
val path = app.root
val path = rootPath / name
val address = app.address + path.toString
val address = path.toString
override def toString = name
@ -163,12 +203,57 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
}
}
// 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)
private val actors = new ConcurrentHashMap[String, AnyRef]
private class Guardian extends Actor {
def receive = {
case Terminated(_) context.self.stop()
}
}
private class SystemGuardian extends Actor {
def receive = {
case Terminated(_)
eventStream.stopDefaultLoggers()
context.self.stop()
}
}
private val guardianFaultHandlingStrategy = {
import akka.actor.FaultHandlingStrategy._
OneForOneStrategy {
case _: ActorKilledException Stop
case _: ActorInitializationException Stop
case _: Exception Restart
}
}
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
// 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)
/*
* 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
private var _guardian: ActorRef = _
@volatile
private var _systemGuardian: ActorRef = _
def guardian = _guardian
def systemGuardian = _systemGuardian
val deathWatch = createDeathWatch()
def init(system: ActorSystemImpl) {
rootGuardian = actorOf(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true)
_guardian = actorOf(system, guardianProps, rootGuardian, "app", true)
_systemGuardian = actorOf(system, 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 "system" name for now)
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] = {
@ -193,12 +278,12 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
*/
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(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(system, props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
private[akka] def actorOf(system: 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
@ -207,7 +292,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
// create a local actor
case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, DeploymentConfig.LocalScope))
new LocalActorRef(app, props, supervisor, path, systemService) // create a local actor
new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
// create a routed actor ref
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.LocalScope))
@ -217,7 +302,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
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, settings.ActorTimeout)
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
@ -226,10 +311,10 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
val connections: Iterable[ActorRef] = (1 to nrOfInstances.factor) map { i
val routedPath = path.parent / (path.name + ":" + i)
new LocalActorRef(app, props, supervisor, routedPath, systemService)
new LocalActorRef(system, props, supervisor, routedPath, systemService)
}
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
actorOf(system, 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)
}
@ -254,7 +339,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
/**
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
*/
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
def actorOf(system: 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
@ -267,34 +352,26 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
// val localOnly = props.localOnly
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
// else new RoutedActorRef(props, address)
new RoutedActorRef(app, props, supervisor, name)
new RoutedActorRef(system, props, supervisor, name)
}
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(rootPath.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.DefaultPromise
(if (within == null) app.AkkaConfig.ActorTimeout else within) match {
(if (within == null) settings.ActorTimeout else within) match {
case t if t.duration.length <= 0
new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout
new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout
case t
val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this.path.toString) }
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
}
}
private[akka] val dummyAskSender = new DeadLetterActorRef(app)
private val tempNumber = new AtomicLong
def tempPath = {
val l = tempNumber.getAndIncrement()
"$_" + Helpers.base64(l)
}
}
class LocalDeathWatch extends DeathWatch with ActorClassification {

View file

@ -6,19 +6,19 @@ package akka.actor
import akka.config._
import akka.actor._
import akka.event._
import akka.dispatch._
import akka.util.duration._
import java.net.InetAddress
import com.eaio.uuid.UUID
import akka.dispatch.{ Dispatchers, Future, Mailbox, Envelope, SystemMessage }
import akka.util.Duration
import akka.util.ReflectiveAccess
import akka.serialization.Serialization
import akka.remote.RemoteAddress
import org.jboss.netty.akka.util.HashedWheelTimer
import java.util.concurrent.{ Executors, TimeUnit }
object ActorSystem {
type AkkaConfig = a.AkkaConfig.type forSome { val a: ActorSystem }
val Version = "2.0-SNAPSHOT"
val envHome = System.getenv("AKKA_HOME") match {
@ -61,26 +61,20 @@ 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
case class Failed(cause: Throwable) extends ExitStatus
}
class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory {
def this(name: String) = this(name, ActorSystem.defaultConfig)
def this() = this("default")
import ActorSystem._
object AkkaConfig {
class Settings(val config: Configuration) {
import config._
val ConfigVersion = getString("akka.version", Version)
@ -126,90 +120,82 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
val FailureDetectorThreshold: Int = getInt("akka.remote.failure-detector.threshold", 8)
val FailureDetectorMaxSampleSize: Int = getInt("akka.remote.failure-detector.max-sample-size", 1000)
}
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true)
import AkkaConfig._
if (ConfigVersion != Version)
throw new ConfigurationException("Akka JAR version [" + Version +
"] does not match the provided config version [" + ConfigVersion + "]")
}
}
abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
import ActorSystem._
def name: String
def settings: Settings
def nodename: String
/**
* Construct a path below the application guardian.
*/
def /(name: String): ActorPath
def rootPath: ActorPath
val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000
val nodename: String = System.getProperty("akka.cluster.nodename") match {
case null | "" new UUID().toString
case value value
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 settings = new Settings(config)
protected def systemImpl = this
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true)
import settings._
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostAddress
case value value
}, System.getProperty("akka.remote.port") match {
case null | "" AkkaConfig.RemoteServerPort
case null | "" settings.RemoteServerPort
case value value.toInt
})
// this provides basic logging (to stdout) until .start() is called below
val eventStream = new EventStream(DebugEventStream)
eventStream.startStdoutLogger(AkkaConfig)
val log = new BusLogging(eventStream, this)
// TODO correctly pull its config from the config
val dispatcherFactory = new Dispatchers(this)
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
def scheduler = provider.scheduler
// TODO think about memory consistency effects when doing funky stuff inside constructor
val reflective = new ReflectiveAccess(this)
eventStream.startStdoutLogger(settings)
val log = new BusLogging(eventStream, this) // this used only for .getClass in tagging messages
/**
* The root actor path for this application.
*/
val root: ActorPath = new RootActorPath(this)
val rootPath: ActorPath = new RootActorPath(address)
// TODO think about memory consistency effects when doing funky stuff inside constructor
val provider: ActorRefProvider = reflective.createProvider
def terminationFuture: Future[ExitStatus] = provider.terminationFuture
private class Guardian extends Actor {
def receive = {
case Terminated(_) context.self.stop()
}
}
private class SystemGuardian extends Actor {
def receive = {
case Terminated(_)
eventStream.stopDefaultLoggers()
context.self.stop()
}
}
private val guardianFaultHandlingStrategy = {
import akka.actor.FaultHandlingStrategy._
OneForOneStrategy {
case _: ActorKilledException Stop
case _: ActorInitializationException Stop
case _: Exception Restart
}
}
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
private val rootGuardian: ActorRef =
provider.actorOf(guardianProps, provider.theOneWhoWalksTheBubblesOfSpaceTime, root, true)
protected[akka] val guardian: ActorRef =
provider.actorOf(guardianProps, rootGuardian, "app", true)
protected[akka] val systemGuardian: ActorRef =
provider.actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
// TODO think about memory consistency effects when doing funky stuff inside constructor
val deadLetters = new DeadLetterActorRef(this)
val deadLetters = new DeadLetterActorRef(eventStream, rootPath / "nul")
val deadLetterMailbox = new Mailbox(null) {
becomeClosed()
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
@ -221,34 +207,70 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
override def numberOfMessages = 0
}
val deathWatch = provider.createDeathWatch()
// FIXME make this configurable
val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
// chain death watchers so that killing guardian stops the application
deathWatch.subscribe(systemGuardian, guardian)
deathWatch.subscribe(rootGuardian, systemGuardian)
// TODO correctly pull its config from the config
val dispatcherFactory = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler))
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
deadLetters.init(dispatcher)
val provider: ActorRefProvider = {
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
case Left(e) throw e
case Right(b) b
}
val arguments = List(
classOf[Settings] -> settings,
classOf[ActorPath] -> rootPath,
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, types, values) match {
case Left(e) throw e
case Right(p) p
}
}
def terminationFuture: Future[ExitStatus] = provider.terminationFuture
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())
@volatile
private var _serialization: Serialization = _
def serialization = _serialization
@volatile
private var _typedActor: TypedActor = _
def typedActor = _typedActor
def /(actorName: String): ActorPath = guardian.path / actorName
def start(): this.type = {
if (_serialization != null) throw new IllegalStateException("cannot initialize ActorSystemImpl twice!")
_serialization = new Serialization(this)
_typedActor = new TypedActor(settings, _serialization)
provider.init(this)
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
eventStream.start(this)
eventStream.startDefaultLoggers(this, AkkaConfig)
eventStream.startDefaultLoggers(this)
this
}
// TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor
val deployer = new Deployer(this)
// TODO think about memory consistency effects when doing funky stuff inside constructor
val typedActor = new TypedActor(this)
// TODO think about memory consistency effects when doing funky stuff inside constructor
val serialization = new Serialization(this)
/**
* Create an actor path under the application supervisor (/app).
*/
def /(actorName: String): ActorPath = guardian.path / actorName
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()
}
terminationFuture.onComplete(_ dispatcher.shutdown())
}

View file

@ -14,14 +14,14 @@ import akka.util.Bootable
*/
trait BootableActorLoaderService extends Bootable {
def app: ActorSystem
def system: ActorSystem
val BOOT_CLASSES = app.AkkaConfig.BootClasses
val BOOT_CLASSES = system.settings.BootClasses
lazy val applicationLoader = createApplicationClassLoader()
protected def createApplicationClassLoader(): Option[ClassLoader] = Some({
if (app.AkkaConfig.Home.isDefined) {
val DEPLOY = app.AkkaConfig.Home.get + "/deploy"
if (system.settings.Home.isDefined) {
val DEPLOY = system.settings.Home.get + "/deploy"
val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) {
System.exit(-1)
@ -59,11 +59,11 @@ trait BootableActorLoaderService extends Bootable {
super.onUnload()
// FIXME shutdown all actors
// app.registry.local.shutdownAll
// system.registry.local.shutdownAll
}
}
/**
* Java API for the default JAX-RS/Mist Initializer
*/
class DefaultBootableActorLoaderService(val app: ActorSystem) extends BootableActorLoaderService
class DefaultBootableActorLoaderService(val system: ActorSystem) extends BootableActorLoaderService

View file

@ -5,9 +5,7 @@
package akka.actor
import collection.immutable.Seq
import java.util.concurrent.ConcurrentHashMap
import akka.event.Logging
import akka.actor.DeploymentConfig._
import akka.AkkaException
@ -15,6 +13,7 @@ import akka.config.{ Configuration, ConfigurationException }
import akka.util.Duration
import java.net.InetSocketAddress
import akka.remote.RemoteAddress
import akka.event.EventStream
trait ActorDeployer {
private[akka] def init(deployments: Seq[Deploy]): Unit
@ -34,10 +33,10 @@ trait ActorDeployer {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Deployer(val app: ActorSystem) extends ActorDeployer {
class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream, val nodename: String) extends ActorDeployer {
val deploymentConfig = new DeploymentConfig(app)
val log = Logging(app.eventStream, this)
val deploymentConfig = new DeploymentConfig(nodename)
val log = Logging(eventStream, this)
val instance: ActorDeployer = {
val deployer = new LocalDeployer()
@ -86,7 +85,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer {
private[akka] def pathsInConfig: List[String] = {
val deploymentPath = "akka.actor.deployment"
app.config.getSection(deploymentPath) match {
settings.config.getSection(deploymentPath) match {
case None Nil
case Some(pathConfig)
pathConfig.map.keySet
@ -98,7 +97,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer {
/**
* Lookup deployment in 'akka.conf' configuration file.
*/
private[akka] def lookupInConfig(path: String, configuration: Configuration = app.config): Option[Deploy] = {
private[akka] def lookupInConfig(path: String, configuration: Configuration = settings.config): Option[Deploy] = {
import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor }
// --------------------------------

View file

@ -217,13 +217,13 @@ object DeploymentConfig {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class DeploymentConfig(val app: ActorSystem) {
class DeploymentConfig(val nodename: String) {
import DeploymentConfig._
case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(app.nodename)), replication: ReplicationScheme = Transient) extends Scope
case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(nodename)), replication: ReplicationScheme = Transient) extends Scope
def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home nodeNameFor(home) == app.nodename)
def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home nodeNameFor(home) == nodename)
def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match {
case Deploy(_, _, _, _, ClusterScope(_, replicationScheme)) Some(replicationScheme)

View file

@ -28,14 +28,14 @@ object FSM {
case object StateTimeout
case class TimeoutMarker(generation: Long)
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: ActorSystem) {
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) {
private var ref: Option[Cancellable] = _
def schedule(actor: ActorRef, timeout: Duration) {
if (repeat) {
ref = Some(app.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit))
ref = Some(system.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit))
} else {
ref = Some(app.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit))
ref = Some(system.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit))
}
}
@ -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.settings.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()
}
@ -290,7 +290,7 @@ private[akka] object IOWorker {
case object Shutdown extends Request
}
private[akka] class IOWorker(app: ActorSystem, ioManager: ActorRef, val bufferSize: Int) {
private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val bufferSize: Int) {
import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT }
import IOWorker._

View file

@ -16,7 +16,7 @@ object TypedActor {
* This class represents a Method call, and has a reference to the Method to be called and the parameters to supply
* It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized
*/
case class MethodCall(app: ActorSystem, method: Method, parameters: Array[AnyRef]) {
case class MethodCall(ser: Serialization, method: Method, parameters: Array[AnyRef]) {
def isOneWay = method.getReturnType == java.lang.Void.TYPE
def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType)
@ -40,7 +40,7 @@ object TypedActor {
case null SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, null, null)
case ps if ps.length == 0 SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array[Serializer.Identifier](), Array[Array[Byte]]())
case ps
val serializers: Array[Serializer] = ps map app.serialization.findSerializerFor
val serializers: Array[Serializer] = ps map ser.findSerializerFor
val serializedParameters: Array[Array[Byte]] = Array.ofDim[Array[Byte]](serializers.length)
for (i 0 until serializers.length)
serializedParameters(i) = serializers(i) toBinary parameters(i) //Mutable for the sake of sanity
@ -57,17 +57,17 @@ object TypedActor {
//TODO implement writeObject and readObject to serialize
//TODO Possible optimization is to special encode the parameter-types to conserve space
private def readResolve(): AnyRef = {
val app = akka.serialization.Serialization.app.value
if (app eq null) throw new IllegalStateException(
val system = akka.serialization.Serialization.system.value
if (system eq null) throw new IllegalStateException(
"Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }")
MethodCall(app, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match {
" Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }")
MethodCall(system.serialization, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match {
case null null
case a if a.length == 0 Array[AnyRef]()
case a
val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity
for (i 0 until a.length) {
deserializedParameters(i) = app.serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i))
deserializedParameters(i) = system.serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i))
}
deserializedParameters
})
@ -101,22 +101,22 @@ object TypedActor {
}
/**
* Returns the akka app (for a TypedActor) when inside a method call in a TypedActor.
* Returns the akka system (for a TypedActor) when inside a method call in a TypedActor.
*/
def app = appReference.get match {
case null throw new IllegalStateException("Calling TypedActor.app outside of a TypedActor implementation method!")
def system = appReference.get match {
case null throw new IllegalStateException("Calling TypedActor.system outside of a TypedActor implementation method!")
case some some
}
/**
* Returns the default dispatcher (for a TypedActor) when inside a method call in a TypedActor.
*/
implicit def dispatcher = app.dispatcher
implicit def dispatcher = system.dispatcher
/**
* Returns the default timeout (for a TypedActor) when inside a method call in a TypedActor.
*/
implicit def timeout = app.AkkaConfig.ActorTimeout
implicit def timeout = system.settings.ActorTimeout
}
trait TypedActorFactory { this: ActorRefFactory
@ -264,7 +264,7 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
*
* TypedActors needs, just like Actors, to be Stopped when they are no longer needed, use TypedActor.stop(proxy)
*/
class TypedActor(val app: ActorSystem) {
class TypedActor(val settings: ActorSystem.Settings, var ser: Serialization) {
import TypedActor.MethodCall
/**
@ -313,7 +313,7 @@ class TypedActor(val app: ActorSystem) {
//Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
val actorVar = new AtomVar[ActorRef](null)
val timeout = props.timeout match {
case Props.`defaultTimeout` app.AkkaConfig.ActorTimeout
case Props.`defaultTimeout` settings.ActorTimeout
case x x
}
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T]
@ -330,7 +330,7 @@ class TypedActor(val app: ActorSystem) {
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 {
@ -365,7 +365,7 @@ class TypedActor(val app: ActorSystem) {
case "equals" (args.length == 1 && (proxy eq args(0)) || actor == getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean
case "hashCode" actor.hashCode.asInstanceOf[AnyRef]
case _
MethodCall(app, method, args) match {
MethodCall(ser, method, args) match {
case m if m.isOneWay actor ! m; null //Null return value
case m if m.returnsFuture_? actor.?(m, timeout)
case m if m.returnsJOption_? || m.returnsOption_?

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.nodename)
def apply(system: ActorSystem): NodeAddress = new NodeAddress(system.settings.ClusterName, system.nodename)
def unapply(other: Any) = other match {
case address: NodeAddress Some((address.clusterName, address.nodeName))

View file

@ -54,7 +54,7 @@ class FilesystemImporter(val baseDir: String) extends Importer {
/**
* An Importer that looks for imported config files in the java resources
* of the system class loader (usually the jar used to launch this app).
* of the system class loader (usually the jar used to launch this system).
*/
class ResourceImporter(classLoader: ClassLoader) extends Importer {
def importFile(filename: String): String = {

View file

@ -14,6 +14,8 @@ import akka.actor._
import akka.actor.ActorSystem
import locks.ReentrantLock
import scala.annotation.tailrec
import akka.event.EventStream
import akka.actor.ActorSystem.Settings
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -62,12 +64,12 @@ case class Supervise(child: ActorRef) extends SystemMessage // sent to superviso
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsWatching
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsWatching
final case class TaskInvocation(app: ActorSystem, function: () Unit, cleanup: () Unit) extends Runnable {
final case class TaskInvocation(eventStream: EventStream, function: () Unit, cleanup: () Unit) extends Runnable {
def run() {
try {
function()
} catch {
case e app.eventStream.publish(Error(e, this, e.getMessage))
case e eventStream.publish(Error(e, this, e.getMessage))
} finally {
cleanup()
}
@ -79,26 +81,23 @@ object MessageDispatcher {
val SCHEDULED = 1
val RESCHEDULED = 2
implicit def defaultDispatcher(implicit app: ActorSystem) = app.dispatcher
implicit def defaultDispatcher(implicit system: ActorSystem) = system.dispatcher
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDispatcher with Serializable {
abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Serializable {
import MessageDispatcher._
import AbstractMessageDispatcher.{ inhabitantsUpdater, shutdownScheduleUpdater }
import prerequisites._
/**
* Creates and returns a mailbox for the given actor.
*/
protected[akka] def createMailbox(actor: ActorCell): Mailbox
/**
* a blackhole mailbox for the purpose of replacing the real one upon actor termination
*/
import app.deadLetterMailbox
/**
* Name of this dispatcher.
*/
@ -119,7 +118,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
}
protected[akka] final def dispatchTask(block: () Unit) {
val invocation = TaskInvocation(app, block, taskCleanup)
val invocation = TaskInvocation(eventStream, block, taskCleanup)
inhabitantsUpdater.incrementAndGet(this)
try {
executeTask(invocation)
@ -136,7 +135,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
shutdownScheduleUpdater.get(this) match {
case UNSCHEDULED
if (shutdownScheduleUpdater.compareAndSet(this, UNSCHEDULED, SCHEDULED)) {
app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
()
} else ifSensibleToDoSoThenScheduleShutdown()
case SCHEDULED
@ -211,7 +210,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
}
case RESCHEDULED
if (shutdownScheduleUpdater.compareAndSet(MessageDispatcher.this, RESCHEDULED, SCHEDULED))
app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
else run()
}
}
@ -289,29 +288,31 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
/**
* Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig
*/
abstract class MessageDispatcherConfigurator(val app: ActorSystem) {
abstract class MessageDispatcherConfigurator() {
/**
* Returns an instance of MessageDispatcher given a Configuration
*/
def configure(config: Configuration): MessageDispatcher
def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher
def mailboxType(config: Configuration): MailboxType = {
val capacity = config.getInt("mailbox-capacity", app.AkkaConfig.MailboxCapacity)
def mailboxType(config: Configuration, settings: Settings): MailboxType = {
val capacity = config.getInt("mailbox-capacity", settings.MailboxCapacity)
if (capacity < 1) UnboundedMailbox()
else {
val duration = Duration(
config.getInt("mailbox-push-timeout-time", app.AkkaConfig.MailboxPushTimeout.toMillis.toInt),
app.AkkaConfig.DefaultTimeUnit)
config.getInt("mailbox-push-timeout-time", settings.MailboxPushTimeout.toMillis.toInt),
settings.DefaultTimeUnit)
BoundedMailbox(capacity, duration)
}
}
def configureThreadPool(config: Configuration, createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
def configureThreadPool(config: Configuration,
settings: Settings,
createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
import ThreadPoolConfigDispatcherBuilder.conf_?
//Apply the following options to the config if they are present in the config
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(app)).configure(
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, app.AkkaConfig.DefaultTimeUnit))),
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()).configure(
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, settings.DefaultTimeUnit))),
conf_?(config getDouble "core-pool-size-factor")(factor _.setCorePoolSizeFromFactor(factor)),
conf_?(config getDouble "max-pool-size-factor")(factor _.setMaxPoolSizeFromFactor(factor)),
conf_?(config getBool "allow-core-timeout")(allow _.setAllowCoreThreadTimeout(allow)),

View file

@ -10,6 +10,8 @@ import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, Concur
import java.util.{ Comparator, Queue }
import annotation.tailrec
import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -28,16 +30,14 @@ import akka.actor.ActorSystem
* @author Viktor Klang
*/
class BalancingDispatcher(
_app: ActorSystem,
_prerequisites: DispatcherPrerequisites,
_name: String,
throughput: Int,
throughputDeadlineTime: Int,
mailboxType: MailboxType,
config: ThreadPoolConfig,
_timeoutMs: Long)
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
import app.deadLetterMailbox
extends Dispatcher(_prerequisites, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
@ -82,7 +82,7 @@ class BalancingDispatcher(
// message must be virgin before being able to systemEnqueue again
val next = message.next
message.next = null
deadLetterMailbox.systemEnqueue(actor.self, message)
prerequisites.deadLetterMailbox.systemEnqueue(actor.self, message)
message = next
}
}

View file

@ -9,6 +9,8 @@ import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue }
import akka.actor.{ ActorCell, ActorKilledException }
import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
/**
* Default settings are:
@ -62,14 +64,14 @@ import akka.actor.ActorSystem
* Larger values (or zero or negative) increase throughput, smaller values increase fairness
*/
class Dispatcher(
_app: ActorSystem,
_prerequisites: DispatcherPrerequisites,
val name: String,
val throughput: Int,
val throughputDeadlineTime: Int,
val mailboxType: MailboxType,
executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
val timeoutMs: Long)
extends MessageDispatcher(_app) {
extends MessageDispatcher(_prerequisites) {
protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name)
protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate {
@ -97,7 +99,7 @@ class Dispatcher(
executorService.get() execute invocation
} catch {
case e2: RejectedExecutionException
app.eventStream.publish(Warning(this, e2.toString))
prerequisites.eventStream.publish(Warning(this, e2.toString))
throw e2
}
}

View file

@ -10,6 +10,20 @@ import akka.util.{ Duration, ReflectiveAccess }
import akka.config.Configuration
import java.util.concurrent.TimeUnit
import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
import akka.actor.ActorSystem.Settings
trait DispatcherPrerequisites {
def eventStream: EventStream
def deadLetterMailbox: Mailbox
def scheduler: Scheduler
}
case class DefaultDispatcherPrerequisites(
val eventStream: EventStream,
val deadLetterMailbox: Mailbox,
val scheduler: Scheduler) extends DispatcherPrerequisites
/**
* Scala API. Dispatcher factory.
@ -41,15 +55,16 @@ import akka.actor.ActorSystem
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Dispatchers(val app: ActorSystem) {
val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: DispatcherPrerequisites) {
val ThroughputDeadlineTimeMillis = settings.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType =
if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(app.AkkaConfig.MailboxCapacity, app.AkkaConfig.MailboxPushTimeout)
val DispatcherShutdownMillis = app.AkkaConfig.DispatcherDefaultShutdown.toMillis
if (settings.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(settings.MailboxCapacity, settings.MailboxPushTimeout)
val DispatcherShutdownMillis = settings.DispatcherDefaultShutdown.toMillis
lazy val defaultGlobalDispatcher =
app.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
settings.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
/**
* Creates an thread based dispatcher serving a single actor through the same single thread.
@ -58,8 +73,8 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(actor: LocalActorRef) = actor match {
case null new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.address, MailboxType, DispatcherShutdownMillis)
case null new PinnedDispatcher(prerequisites, null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(prerequisites, some.underlying, some.address, MailboxType, DispatcherShutdownMillis)
}
/**
@ -69,8 +84,8 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
case null new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.address, mailboxType, DispatcherShutdownMillis)
case null new PinnedDispatcher(prerequisites, null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(prerequisites, some.underlying, some.address, mailboxType, DispatcherShutdownMillis)
}
/**
@ -79,7 +94,7 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(name: String, mailboxType: MailboxType) =
new PinnedDispatcher(app, null, name, mailboxType, DispatcherShutdownMillis)
new PinnedDispatcher(prerequisites, null, name, mailboxType, DispatcherShutdownMillis)
/**
* Creates an thread based dispatcher serving a single actor through the same single thread.
@ -87,7 +102,7 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread.
*/
def newPinnedDispatcher(name: String) =
new PinnedDispatcher(app, null, name, MailboxType, DispatcherShutdownMillis)
new PinnedDispatcher(prerequisites, null, name, MailboxType, DispatcherShutdownMillis)
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -95,8 +110,8 @@ class Dispatchers(val app: ActorSystem) {
* Has a fluent builder interface for configuring its semantics.
*/
def newDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(app, name, app.AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(prerequisites, name, settings.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -105,7 +120,7 @@ class Dispatchers(val app: ActorSystem) {
*/
def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
new Dispatcher(prerequisites, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -114,7 +129,7 @@ class Dispatchers(val app: ActorSystem) {
*/
def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
new Dispatcher(prerequisites, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -122,8 +137,8 @@ class Dispatchers(val app: ActorSystem) {
* Has a fluent builder interface for configuring its semantics.
*/
def newBalancingDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(app, name, app.AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(prerequisites, name, settings.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -132,7 +147,7 @@ class Dispatchers(val app: ActorSystem) {
*/
def newBalancingDispatcher(name: String, throughput: Int) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
new BalancingDispatcher(prerequisites, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -141,7 +156,7 @@ class Dispatchers(val app: ActorSystem) {
*/
def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
new BalancingDispatcher(prerequisites, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -150,13 +165,13 @@ class Dispatchers(val app: ActorSystem) {
*/
def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
new BalancingDispatcher(prerequisites, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/**
* Utility function that tries to load the specified dispatcher config from the akka.conf
* or else use the supplied default dispatcher
*/
def fromConfig(key: String, default: MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher =
app.config getSection key flatMap from getOrElse default
settings.config getSection key flatMap from getOrElse default
/*
* Creates of obtains a dispatcher from a ConfigMap according to the format below
@ -181,9 +196,8 @@ class Dispatchers(val app: ActorSystem) {
*/
def from(cfg: Configuration): Option[MessageDispatcher] = {
cfg.getString("type") flatMap {
case "Dispatcher" Some(new DispatcherConfigurator(app))
case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator(app))
case "GlobalDispatcher" None //TODO FIXME remove this
case "Dispatcher" Some(new DispatcherConfigurator())
case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator())
case fqn
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
case Right(clazz)
@ -197,31 +211,35 @@ class Dispatchers(val app: ActorSystem) {
throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn, exception)
}
} map {
_ configure cfg
_.configure(cfg, settings, prerequisites)
}
}
}
class DispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) {
def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new Dispatcher(app,
class DispatcherConfigurator() extends MessageDispatcherConfigurator() {
def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
configureThreadPool(config,
settings,
threadPoolConfig new Dispatcher(prerequisites,
config.getString("name", newUuid.toString),
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
config.getInt("throughput", settings.DispatcherThroughput),
config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config, settings),
threadPoolConfig,
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
settings.DispatcherDefaultShutdown.toMillis)).build
}
}
class BalancingDispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) {
def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(app,
class BalancingDispatcherConfigurator() extends MessageDispatcherConfigurator() {
def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
configureThreadPool(config,
settings,
threadPoolConfig new BalancingDispatcher(prerequisites,
config.getString("name", newUuid.toString),
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config),
config.getInt("throughput", settings.DispatcherThroughput),
config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config, settings),
threadPoolConfig,
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
settings.DispatcherDefaultShutdown.toMillis)).build
}
}

View file

@ -262,7 +262,7 @@ object Future {
result completeWithResult currentValue
} catch {
case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
result completeWithException e
} finally {
results.clear
@ -631,7 +631,7 @@ sealed trait Future[+T] extends japi.Future[T] {
Right(f(res))
} catch {
case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
Left(e)
})
}
@ -683,7 +683,7 @@ sealed trait Future[+T] extends japi.Future[T] {
future.completeWith(f(r))
} catch {
case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
future complete Left(e)
}
}
@ -716,7 +716,7 @@ sealed trait Future[+T] extends japi.Future[T] {
if (p(res)) r else Left(new MatchError(res))
} catch {
case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
Left(e)
})
}
@ -788,7 +788,7 @@ trait Promise[T] extends Future[T] {
fr completeWith cont(f)
} catch {
case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
fr completeWithException e
}
}
@ -802,7 +802,7 @@ trait Promise[T] extends Future[T] {
fr completeWith cont(f)
} catch {
case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
fr completeWithException e
}
}
@ -956,12 +956,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
val runnable = new Runnable {
def run() {
if (!isCompleted) {
if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
if (!isExpired) dispatcher.prerequisites.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
else func(DefaultPromise.this)
}
}
}
val timeoutFuture = dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
val timeoutFuture = dispatcher.prerequisites.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
onComplete(_ timeoutFuture.cancel())
false
} else true
@ -983,18 +983,18 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
val runnable = new Runnable {
def run() {
if (!isCompleted) {
if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
if (!isExpired) dispatcher.prerequisites.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
else promise complete (try { Right(fallback) } catch { case e Left(e) })
}
}
}
dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
dispatcher.prerequisites.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
promise
}
} else this
private def notifyCompleted(func: Future[T] Unit) {
try { func(this) } catch { case e dispatcher.app.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really?
try { func(this) } catch { case e dispatcher.prerequisites.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really?
}
@inline

View file

@ -187,7 +187,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
}
} catch {
case e
actor.app.eventStream.publish(Error(e, actor.self, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
actor.system.eventStream.publish(Error(e, actor.self, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
throw e
}
}

View file

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

View file

@ -10,6 +10,7 @@ import akka.util.Duration
import akka.event.Logging.{ Warning, Error }
import akka.actor.ActorSystem
import java.util.concurrent._
import akka.event.EventStream
object ThreadPoolConfig {
type Bounds = Int
@ -63,8 +64,7 @@ trait ExecutorServiceFactoryProvider {
/**
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
*/
case class ThreadPoolConfig(app: ActorSystem,
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,

View file

@ -3,9 +3,7 @@
*/
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
class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification {
@ -41,8 +39,8 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas
super.unsubscribe(subscriber)
}
def start(app: ActorSystem) {
reaper = app.systemActorOf(Props(new Actor {
def start(system: ActorSystemImpl) {
reaper = system.systemActorOf(Props(new Actor {
def receive = {
case ref: ActorRef watch(ref)
case Terminated(ref) unsubscribe(ref)

View file

@ -3,9 +3,9 @@
*/
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.actor.ActorSystem.Settings
import akka.util.ReflectiveAccess
import akka.config.ConfigurationException
import akka.util.ReentrantGuard
@ -13,6 +13,7 @@ import akka.util.duration._
import akka.actor.Timeout
import akka.dispatch.FutureTimeoutException
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.ActorRefProvider
/**
* This trait brings log level handling to the EventStream: it reads the log
@ -65,7 +66,7 @@ trait LoggingBus extends ActorEventBus {
_logLevel = level
}
private[akka] def startStdoutLogger(config: AkkaConfig) {
private[akka] def startStdoutLogger(config: Settings) {
val level = levelFor(config.StdoutLogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.StdoutLogLevel))
ErrorLevel
@ -78,13 +79,13 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "StandardOutLogger started"))
}
private[akka] def startDefaultLoggers(app: ActorSystem, config: AkkaConfig) {
val level = levelFor(config.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel))
private[akka] def startDefaultLoggers(system: ActorSystemImpl) {
val level = levelFor(system.settings.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + system.settings.LogLevel))
ErrorLevel
}
try {
val defaultLoggers = config.EventHandlers match {
val defaultLoggers = system.settings.EventHandlers match {
case Nil "akka.event.Logging$DefaultLogger" :: Nil
case loggers loggers
}
@ -94,7 +95,7 @@ trait LoggingBus extends ActorEventBus {
} yield {
try {
ReflectiveAccess.getClassFor[Actor](loggerName) match {
case Right(actorClass) addLogger(app, actorClass, level)
case Right(actorClass) addLogger(system, actorClass, level)
case Left(exception) throw exception
}
} catch {
@ -137,9 +138,9 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "all default loggers stopped"))
}
private def addLogger(app: ActorSystem, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz)
val actor = app.systemActorOf(Props(clazz), name)
val actor = system.systemActorOf(Props(clazz), name)
implicit val timeout = Timeout(3 seconds)
val response = try actor ? InitializeLogger(this) get catch {
case _: FutureTimeoutException
@ -236,12 +237,12 @@ object Logging {
* Obtain LoggingAdapter for the given application and source object. The
* source object is used to identify the source of this logging channel.
*/
def apply(app: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(app.eventStream, source)
def apply(system: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(system.eventStream, source)
/**
* Java API: Obtain LoggingAdapter for the given application and source object. The
* source object is used to identify the source of this logging channel.
*/
def getLogger(app: ActorSystem, source: AnyRef): LoggingAdapter = apply(app, source)
def getLogger(system: ActorSystem, source: AnyRef): LoggingAdapter = apply(system, source)
/**
* Obtain LoggingAdapter for the given event bus and source object. The
* source object is used to identify the source of this logging channel.

View file

@ -128,7 +128,7 @@ case class CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorExcept
override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter)
}
abstract class RemoteSupport(val app: ActorSystem) {
abstract class RemoteSupport(val system: ActorSystem) {
/**
* Shuts down the remoting
*/
@ -162,7 +162,7 @@ abstract class RemoteSupport(val app: ActorSystem) {
recipient: ActorRef,
loader: Option[ClassLoader]): Unit
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = app.eventStream.publish(message)
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = system.eventStream.publish(message)
override def toString = name
}

View file

@ -155,7 +155,7 @@ object Routing {
/**
* An Abstract convenience implementation for building an ActorReference that uses a Router.
*/
abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val props: RoutedProps) extends MinimalActorRef {
abstract private[akka] class AbstractRoutedActorRef(val system: ActorSystem, val props: RoutedProps) extends MinimalActorRef {
val router = props.routerFactory()
override def !(message: Any)(implicit sender: ActorRef = null): Unit = router.route(message)(sender)
@ -167,7 +167,7 @@ abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val pr
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors.
*/
private[akka] class RoutedActorRef(app: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(app, routedProps) {
private[akka] class RoutedActorRef(system: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(system, routedProps) {
val path = supervisor.path / name

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 system: ActorSystemImpl) {
//TODO document me
def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
@ -28,7 +28,7 @@ class Serialization(val app: ActorSystem) {
clazz: Class[_],
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
try {
Serialization.app.withValue(app) {
Serialization.system.withValue(system) {
Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader))
}
} catch { case e: Exception Left(e) }
@ -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")
system.settings.config.getSection("akka.actor.serializers")
.map(_.map)
.getOrElse(Map())
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) {
@ -81,7 +81,7 @@ class Serialization(val app: 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] = system.settings.config.getSection("akka.actor.serialization-bindings") map {
_.map.foldLeft(Map[String, String]()) {
case (result, (k: String, vs: List[_])) result ++ (vs collect { case v: String (v, k) }) //All keys which are lists, take the Strings from them and Map them
case (result, _) result //For any other values, just skip them, TODO: print out warnings?
@ -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 system = new DynamicVariable[ActorSystemImpl](null)
}

View file

@ -8,7 +8,7 @@ import akka.actor.ActorSystem
/*
* This class is responsible for booting up a stack of bundles and then shutting them down
*/
class AkkaLoader(app: ActorSystem) {
class AkkaLoader(system: ActorSystem) {
private val hasBooted = new Switch(false)
@volatile

View file

@ -278,7 +278,7 @@ abstract class Duration extends Serializable {
def /(other: Duration): Double
def unary_- : Duration
def finite_? : Boolean
def dilated(implicit app: ActorSystem): Duration = this * app.AkkaConfig.TestTimeFactor
def dilated(implicit system: ActorSystem): Duration = this * system.settings.TestTimeFactor
def min(other: Duration): Duration = if (this < other) this else other
def max(other: Duration): Duration = if (this > other) this else other
def sleep(): Unit = Thread.sleep(toMillis)

View file

@ -18,20 +18,20 @@ object JMX {
def nameFor(hostname: String, service: String, bean: String): ObjectName =
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
def register(name: ObjectName, mbean: AnyRef)(implicit app: ActorSystem): Option[ObjectInstance] = try {
def register(name: ObjectName, mbean: AnyRef)(implicit system: ActorSystem): Option[ObjectInstance] = try {
Some(mbeanServer.registerMBean(mbean, name))
} catch {
case e: InstanceAlreadyExistsException
Some(mbeanServer.getObjectInstance(name))
case e: Exception
app.eventStream.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean)))
system.eventStream.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean)))
None
}
def unregister(mbean: ObjectName)(implicit app: ActorSystem) = try {
def unregister(mbean: ObjectName)(implicit system: ActorSystem) = try {
mbeanServer.unregisterMBean(mbean)
} catch {
case e: InstanceNotFoundException {}
case e: Exception app.eventStream.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean)))
case e: Exception system.eventStream.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean)))
}
}

View file

@ -112,29 +112,3 @@ object ReflectiveAccess {
}
/**
* Helper class for reflective access to different modules in order to allow optional loading of modules.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class ReflectiveAccess(val app: ActorSystem) {
import ReflectiveAccess._
def providerClass: Class[_] = {
getClassFor(app.AkkaConfig.ProviderClass) match {
case Left(e) throw e
case Right(b) b
}
}
def createProvider: ActorRefProvider = {
val params: Array[Class[_]] = Array(classOf[ActorSystem])
val args: Array[AnyRef] = Array(app)
createInstance[ActorRefProvider](providerClass, params, args) match {
case Right(p) p
case Left(e) throw e
}
}
}

View file

@ -58,7 +58,7 @@ trait CamelService extends Bootable {
* Starts this CamelService.
*/
def start: CamelService = {
// Only init and start if not already done by app
// Only init and start if not already done by system
if (!CamelContextManager.initialized) CamelContextManager.init
if (!CamelContextManager.started) CamelContextManager.start

View file

@ -35,7 +35,7 @@ trait Consumer { this: Actor ⇒
/**
* Determines whether one-way communications between an endpoint and this consumer actor
* should be auto-acknowledged or app-acknowledged.
* should be auto-acknowledged or system-acknowledged.
*/
def autoack = true
@ -79,7 +79,7 @@ abstract class UntypedConsumerActor extends UntypedActor with Consumer {
/**
* Determines whether one-way communications between an endpoint and this consumer actor
* should be auto-acknowledged or app-acknowledged.
* should be auto-acknowledged or system-acknowledged.
*/
def isAutoack() = super.autoack
}

View file

@ -208,7 +208,7 @@ object Message {
}
/**
* Positive acknowledgement message (used for app-acknowledged message receipts).
* Positive acknowledgement message (used for system-acknowledged message receipts).
*
* @author Martin Krasser
*/

View file

@ -50,7 +50,7 @@ trait ProducerSupport { this: Actor ⇒
/**
* Returns the names of message headers to copy from a request message to a response message.
* By default only the Message.MessageExchangeId is copied. Applications may override this to
* define an app-specific set of message headers to copy.
* define an system-specific set of message headers to copy.
*/
def headersToCopy: Set[String] = headersToCopyDefault

View file

@ -138,15 +138,15 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
"An non auto-acknowledging consumer" when {
"started" must {
"must support acknowledgements on app level" in {
"must support acknowledgements on system level" in {
var consumer: ActorRef = null
service.awaitEndpointActivation(1) {
consumer = actorOf(new TestAckConsumer("direct:app-ack-test"))
consumer = actorOf(new TestAckConsumer("direct:system-ack-test"))
} must be(true)
val endpoint = mandatoryContext.getEndpoint("direct:app-ack-test", classOf[DirectEndpoint])
val endpoint = mandatoryContext.getEndpoint("direct:system-ack-test", classOf[DirectEndpoint])
val producer = endpoint.createProducer.asInstanceOf[AsyncProcessor]
val exchange = endpoint.createExchange

View file

@ -9,13 +9,13 @@ import java.io.File
/*
A simple use of BookKeeper is to implement a write-ahead transaction log. A server maintains an in-memory data structure
(with periodic snapshots for example) and logs changes to that structure before it applies the change. The app
(with periodic snapshots for example) and logs changes to that structure before it applies the change. The system
server creates a ledger at startup and store the ledger id and password in a well known place (ZooKeeper maybe). When
it needs to make a change, the server adds an entry with the change information to a ledger and apply the change when
BookKeeper adds the entry successfully. The server can even use asyncAddEntry to queue up many changes for high change
throughput. BooKeeper meticulously logs the changes in order and call the completion functions in order.
When the app server dies, a backup server will come online, get the last snapshot and then it will open the
When the system server dies, a backup server will come online, get the last snapshot and then it will open the
ledger of the old server and read all the entries from the time the snapshot was taken. (Since it doesn't know the last
entry number it will use MAX_INTEGER). Once all the entries have been processed, it will close the ledger and start a
new one for its use.

View file

@ -10,7 +10,7 @@ import System.{currentTimeMillis => now}
import java.util.concurrent.CountDownLatch
//#imports
//#app
//#system
object Pi extends App {
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
@ -127,5 +127,5 @@ object Pi extends App {
latch.await()
}
}
//#app
//#system

View file

@ -11,10 +11,10 @@
// import java.util.concurrent.CountDownLatch
// //#imports
// //#app
// //#system
// object Pi extends App {
// val app = ActorSystem()
// val system = ActorSystem()
// calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
@ -66,10 +66,10 @@
// //#create-workers
// // create the workers
// val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker])
// val workers = Vector.fill(nrOfWorkers)(system.actorOf[Worker])
// // wrap them with a load-balancing router
// val router = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
// val router = system.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
// //#create-workers
// //#master-receive
@ -119,7 +119,7 @@
// val latch = new CountDownLatch(1)
// // create the master
// val master = app.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
// val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
// // start the calculation
// master ! Calculate
@ -128,5 +128,5 @@
// latch.await()
// }
// }
// //#app
// //#system

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

@ -20,15 +20,15 @@ class BeanstalkBasedMailboxException(message: String) extends AkkaException(mess
*/
class BeanstalkBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
val hostname = app.config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0")
val port = app.config.getInt("akka.actor.mailbox.beanstalk.port", 11300)
def defaultTimeUnit = app.AkkaConfig.DefaultTimeUnit
val reconnectWindow = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), defaultTimeUnit).toSeconds.toInt
val messageSubmitDelay = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.message-submit-delay", 0), defaultTimeUnit).toSeconds.toInt
val messageSubmitTimeout = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.message-submit-timeout", 5), defaultTimeUnit).toSeconds.toInt
val messageTimeToLive = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.message-time-to-live", 120), defaultTimeUnit).toSeconds.toInt
val hostname = system.settings.config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0")
val port = system.settings.config.getInt("akka.actor.mailbox.beanstalk.port", 11300)
def defaultTimeUnit = system.settings.DefaultTimeUnit
val reconnectWindow = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), defaultTimeUnit).toSeconds.toInt
val messageSubmitDelay = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.message-submit-delay", 0), defaultTimeUnit).toSeconds.toInt
val messageSubmitTimeout = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.message-submit-timeout", 5), defaultTimeUnit).toSeconds.toInt
val messageTimeToLive = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.message-time-to-live", 120), defaultTimeUnit).toSeconds.toInt
val log = Logging(app, this)
val log = Logging(system, this)
private val queue = new ThreadLocal[Client] { override def initialValue = connect(name) }

View file

@ -19,13 +19,13 @@ object FileBasedMailbox {
class FileBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
val log = Logging(app, this)
val log = Logging(system, this)
val queuePath = FileBasedMailbox.queuePath(owner.app.config)
val queuePath = FileBasedMailbox.queuePath(owner.system.settings.config)
private val queue = try {
try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case e {} }
val queue = new filequeue.PersistentQueue(queuePath, name, owner.app.config, log)
val queue = new filequeue.PersistentQueue(queuePath, name, owner.system.settings.config, log)
queue.setup // replays journal
queue.discardExpired
queue

View file

@ -142,11 +142,11 @@ object QDumper {
System.exit(0)
}
val app = ActorSystem()
val system = ActorSystem()
for (filename filenames) {
println("Queue: " + filename)
new QueueDumper(filename, app.log)()
new QueueDumper(filename, system.log)()
}
}
}

View file

@ -6,7 +6,7 @@ import org.apache.commons.io.FileUtils
class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileDurableMailboxType) {
def clean {
val queuePath = FileBasedMailbox.queuePath(app.config)
val queuePath = FileBasedMailbox.queuePath(system.settings.config)
FileUtils.deleteDirectory(new java.io.File(queuePath))
}

View file

@ -40,7 +40,7 @@ class DurableMailboxException private[akka] (message: String, cause: Throwable)
abstract class DurableMailbox(owner: ActorCell) extends Mailbox(owner) with DefaultSystemMessageQueue {
import DurableExecutableMailboxConfig._
def app = owner.app
def system = owner.system
def ownerPath = owner.self.path
val ownerPathString = ownerPath.path.mkString("/")
val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
@ -54,11 +54,11 @@ trait DurableMessageSerialization {
def serialize(durableMessage: Envelope): Array[Byte] = {
def serializeActorRef(ref: ActorRef): ActorRefProtocol = {
val serRef = owner.app.provider.serialize(ref)
val serRef = owner.system.provider.serialize(ref)
ActorRefProtocol.newBuilder.setPath(serRef.path).setHost(serRef.hostname).setPort(serRef.port).build
}
val message = MessageSerializer.serialize(owner.app, durableMessage.message.asInstanceOf[AnyRef])
val message = MessageSerializer.serialize(owner.system, durableMessage.message.asInstanceOf[AnyRef])
val builder = RemoteMessageProtocol.newBuilder
.setMessage(message)
.setRecipient(serializeActorRef(owner.self))
@ -71,11 +71,11 @@ trait DurableMessageSerialization {
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = {
val serRef = SerializedActorRef(refProtocol.getHost, refProtocol.getPort, refProtocol.getPath)
owner.app.provider.deserialize(serRef).getOrElse(owner.app.deadLetters)
owner.system.provider.deserialize(serRef).getOrElse(owner.system.deadLetters)
}
val durableMessage = RemoteMessageProtocol.parseFrom(bytes)
val message = MessageSerializer.deserialize(owner.app, durableMessage.getMessage)
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage)
val sender = deserializeActorRef(durableMessage.getSender)
new Envelope(message, sender)

View file

@ -26,7 +26,7 @@ object DurableMailboxSpecActorFactory {
abstract class DurableMailboxSpec(val backendName: String, val mailboxType: DurableMailboxType) extends AkkaSpec with BeforeAndAfterEach {
import DurableMailboxSpecActorFactory._
implicit val dispatcher = new Dispatchers(app).newDispatcher(backendName, throughput = 1, mailboxType = mailboxType).build
implicit val dispatcher = system.dispatcherFactory.newDispatcher(backendName, throughput = 1, mailboxType = mailboxType).build
def createMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef =
actorOf(Props(new MailboxTestActor).withDispatcher(dispatcher))

View file

@ -17,9 +17,11 @@ import org.bson.DefaultBSONSerializer
import akka.actor.SerializedActorRef
import akka.remote.RemoteProtocol.MessageProtocol
import akka.remote.MessageSerializer
import akka.actor.ActorSystem
import akka.actor.{ ActorSystem, ActorSystemImpl }
class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
val systemImpl = system.asInstanceOf[ActorSystemImpl]
protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = {
@ -28,7 +30,7 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
b += "_id" -> msg._id
b += "ownerPath" -> msg.ownerPath
val sender = app.provider.serialize(msg.sender)
val sender = systemImpl.provider.serialize(msg.sender)
b += "senderPath" -> sender.path
b += "senderHostname" -> sender.hostname
b += "senderPort" -> sender.port
@ -37,10 +39,10 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
* TODO - Figure out a way for custom serialization of the message instance
* TODO - Test if a serializer is registered for the message and if not, use toByteString
*/
val msgData = MessageSerializer.serialize(app, msg.message.asInstanceOf[AnyRef])
val msgData = MessageSerializer.serialize(system, msg.message.asInstanceOf[AnyRef])
b += "message" -> new org.bson.types.Binary(0, msgData.toByteArray)
val doc = b.result
app.log.debug("Serialized Document: {}", doc)
system.log.debug("Serialized Document: {}", doc)
serializer.putObject(doc)
}
@ -68,15 +70,15 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
val deserializer = new DefaultBSONDeserializer
// TODO - Skip the whole doc step for performance, fun, and profit! (Needs Salat / custom Deser)
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
app.log.debug("Deserializing a durable message from MongoDB: {}", doc)
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
val msg = MessageSerializer.deserialize(app, msgData)
val msg = MessageSerializer.deserialize(system, msgData)
val ownerPath = doc.as[String]("ownerPath")
val senderPath = doc.as[String]("senderPath")
val senderHostname = doc.as[String]("senderHostname")
val senderPort = doc.as[Int]("senderPort")
val sender = app.provider.deserialize(SerializedActorRef(senderHostname, senderPort, senderPath)).
getOrElse(app.deadLetters)
val sender = systemImpl.provider.deserialize(SerializedActorRef(senderHostname, senderPort, senderPath)).
getOrElse(system.deadLetters)
MongoDurableMessage(ownerPath, msg, sender)
}

View file

@ -28,17 +28,17 @@ class MongoBasedMailboxException(message: String) extends AkkaException(message)
*/
class MongoBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) {
// this implicit object provides the context for reading/writing things as MongoDurableMessage
implicit val mailboxBSONSer = new BSONSerializableMailbox(app)
implicit val mailboxBSONSer = new BSONSerializableMailbox(system)
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
val URI_CONFIG_KEY = "akka.actor.mailbox.mongodb.uri"
val WRITE_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.write"
val READ_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.read"
val mongoURI = app.config.getString(URI_CONFIG_KEY)
val writeTimeout = app.config.getInt(WRITE_TIMEOUT_KEY, 3000)
val readTimeout = app.config.getInt(READ_TIMEOUT_KEY, 3000)
val mongoURI = system.settings.config.getString(URI_CONFIG_KEY)
val writeTimeout = system.settings.config.getInt(WRITE_TIMEOUT_KEY, 3000)
val readTimeout = system.settings.config.getInt(READ_TIMEOUT_KEY, 3000)
val log = Logging(app, this)
val log = Logging(system, this)
@volatile
private var mongo = connect()

View file

@ -20,7 +20,7 @@ class RedisBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with
@volatile
private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling
val log = Logging(app, this)
val log = Logging(system, this)
def enqueue(receiver: ActorRef, envelope: Envelope) {
log.debug("ENQUEUING message in redis-based mailbox [%s]".format(envelope))
@ -58,8 +58,8 @@ class RedisBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with
private[akka] def connect() = {
new RedisClientPool(
app.config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"),
app.config.getInt("akka.actor.mailbox.redis.port", 6379))
system.settings.config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"),
system.settings.config.getInt("akka.actor.mailbox.redis.port", 6379))
}
private def withErrorHandling[T](body: T): T = {

View file

@ -21,16 +21,16 @@ class ZooKeeperBasedMailboxException(message: String) extends AkkaException(mess
*/
class ZooKeeperBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
val zkServerAddresses = app.config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181")
def defaultTimeUnit = app.AkkaConfig.DefaultTimeUnit
val sessionTimeout = Duration(app.config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), defaultTimeUnit).toMillis.toInt
val connectionTimeout = Duration(app.config.getInt("akka.actor.mailbox.zookeeper.connection-timeout", 60), defaultTimeUnit).toMillis.toInt
val blockingQueue = app.config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true)
val zkServerAddresses = system.settings.config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181")
def defaultTimeUnit = system.settings.DefaultTimeUnit
val sessionTimeout = Duration(system.settings.config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), defaultTimeUnit).toMillis.toInt
val connectionTimeout = Duration(system.settings.config.getInt("akka.actor.mailbox.zookeeper.connection-timeout", 60), defaultTimeUnit).toMillis.toInt
val blockingQueue = system.settings.config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true)
val queueNode = "/queues"
val queuePathTemplate = queueNode + "/%s"
val log = Logging(app, this)
val log = Logging(system, this)
private val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout)
private val queue = new ZooKeeperQueue[Array[Byte]](zkClient, queuePathTemplate.format(name), blockingQueue)

View file

@ -14,13 +14,13 @@ import javax.servlet.{ ServletContextListener, ServletContextEvent }
/**
* This class can be added to web.xml mappings as a listener to start and postStop Akka.
*
* <web-app>
* <web-system>
* ...
* <listener>
* <listener-class>akka.servlet.Initializer</listener-class>
* </listener>
* ...
* </web-app>
* </web-system>
*/
class Initializer extends ServletContextListener {
lazy val loader = new AkkaLoader

View file

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

View file

@ -21,26 +21,26 @@ trait BootableRemoteActorService extends Bootable {
def settings: RemoteServerSettings
protected lazy val remoteServerThread = new Thread(new Runnable() {
def run = app.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port
def run = system.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port
}, "Akka RemoteModule Service")
def startRemoteService() { remoteServerThread.start() }
abstract override def onLoad() {
if (app.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
app.eventHandler.info(this, "Initializing Remote Actors Service...")
if (system.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
system.eventHandler.info(this, "Initializing Remote Actors Service...")
startRemoteService()
app.eventHandler.info(this, "Remote Actors Service initialized")
system.eventHandler.info(this, "Remote Actors Service initialized")
}
super.onLoad()
}
abstract override def onUnload() {
app.eventHandler.info(this, "Shutting down Remote Actors Service")
system.eventHandler.info(this, "Shutting down Remote Actors Service")
app.remote.shutdown()
system.remote.shutdown()
if (remoteServerThread.isAlive) remoteServerThread.join(1000)
app.eventHandler.info(this, "Remote Actors Service has been shut down")
system.eventHandler.info(this, "Remote Actors Service has been shut down")
super.onUnload()
}
}

View file

@ -101,13 +101,13 @@ class Gossiper(remote: Remote) {
currentGossip: Gossip,
nodeMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener])
private val app = remote.app
private val log = Logging(app, this)
private val system = remote.system
private val log = Logging(system, this)
private val failureDetector = remote.failureDetector
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
private val connectionManager = new RemoteConnectionManager(system, 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 = system.rootPath.remoteAddress
private val nodeFingerprint = address.##
private val random = SecureRandom.getInstance("SHA1PRNG")
@ -122,8 +122,8 @@ class Gossiper(remote: Remote) {
{
// start periodic gossip and cluster scrutinization - default is run them every second with 1/2 second in between
app.scheduler schedule (() initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
app.scheduler schedule (() scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
system.scheduler schedule (() initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
system.scheduler schedule (() scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
}
/**
@ -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.system.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
}
@ -299,7 +299,7 @@ class Gossiper(remote: Remote) {
}
private def toRemoteMessage(gossip: Gossip): RemoteProtocol.RemoteSystemDaemonMessageProtocol = {
val gossipAsBytes = app.serialization.serialize(gossip) match {
val gossipAsBytes = system.serialization.serialize(gossip) match {
case Left(error) throw error
case Right(bytes) bytes
}

View file

@ -11,15 +11,15 @@ import akka.actor.ActorSystem
object MessageSerializer {
def deserialize(app: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
val clazz = loadManifest(classLoader, messageProtocol)
app.serialization.deserialize(messageProtocol.getMessage.toByteArray,
system.serialization.deserialize(messageProtocol.getMessage.toByteArray,
clazz, classLoader).fold(x throw x, identity)
}
def serialize(app: ActorSystem, message: AnyRef): MessageProtocol = {
def serialize(system: ActorSystem, message: AnyRef): MessageProtocol = {
val builder = MessageProtocol.newBuilder
val bytes = app.serialization.serialize(message).fold(x throw x, identity)
val bytes = system.serialization.serialize(message).fold(x throw x, identity)
builder.setMessage(ByteString.copyFrom(bytes))
builder.setMessageManifest(ByteString.copyFromUtf8(message.getClass.getName))
builder.build

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,14 +58,14 @@ object NetworkEventStream {
}
}
class NetworkEventStream(val app: ActorSystem) {
class NetworkEventStream(system: ActorSystemImpl) {
import NetworkEventStream._
// FIXME: check that this supervision is correct
private[akka] val sender = app.provider.actorOf(
Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
app.systemGuardian, "network-event-sender", systemService = true)
private[akka] val sender = system.provider.actorOf(system,
Props[Channel].copy(dispatcher = system.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
system.systemGuardian, "network-event-sender", systemService = true)
/**
* Registers a network event stream listener (asyncronously).

View file

@ -15,33 +15,30 @@ 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) {
class Remote(val system: ActorSystemImpl, val nodename: String) {
val log = Logging(app, this)
val log = Logging(system, this)
import app._
import app.config
import app.AkkaConfig._
import system._
val AC = settings
import AC._
val nodename = app.nodename
// TODO move to AkkaConfig?
// TODO move to settings?
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt
val failureDetector = new AccrualFailureDetector(app)
val failureDetector = new AccrualFailureDetector(system)
// val gossiper = new Gossiper(this)
@ -50,17 +47,18 @@ class Remote(val app: ActorSystem) {
// FIXME configure computeGridDispatcher to what?
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
// FIXME it is probably better to create another supervisor for handling the children created by handle_*
private[remote] lazy val remoteDaemonSupervisor = system.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(
system.provider.actorOf(system,
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
remoteDaemonSupervisor,
remoteDaemonServiceName,
systemService = true)
private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor {
private[remote] lazy val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor {
def receive = {
case RemoteClientError(cause, remote, address) remote.shutdownClientConnection(address)
case RemoteClientDisconnected(remote, address) remote.shutdownClientConnection(address)
@ -68,22 +66,22 @@ class Remote(val app: ActorSystem) {
}
}), "akka.remote.RemoteClientLifeCycleListener")
lazy val eventStream = new NetworkEventStream(app)
lazy val eventStream = new NetworkEventStream(system)
lazy val server: RemoteSupport = {
val remote = new akka.remote.netty.NettyRemoteSupport(app)
val remote = new akka.remote.netty.NettyRemoteSupport(system)
remote.start() //TODO FIXME Any application loader here?
app.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
app.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
// TODO actually register this provider in app in remote mode
// TODO actually register this provider in system in remote mode
//provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
remote
}
def start(): Unit = {
val serverAddress = server.app.address //Force init of server
val serverAddress = server.system.rootPath.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)
}
@ -99,6 +97,7 @@ class Remote(val app: ActorSystem) {
class RemoteSystemDaemon(remote: Remote) extends Actor {
import remote._
import remote.{ system systemImpl }
override def preRestart(reason: Throwable, msg: Option[Any]) {
log.debug("RemoteSystemDaemon failed due to [{}] - restarting...", reason)
@ -135,16 +134,16 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
if (shouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
val actorFactory =
app.serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match {
system.serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match {
case Left(error) throw error
case Right(instance) instance.asInstanceOf[() Actor]
}
val actorPath = ActorPath(remote.app, message.getActorPath)
val parent = actorPath.parent.ref
val actorPath = ActorPath(systemImpl, message.getActorPath)
val parent = system.actorFor(actorPath.parent)
if (parent.isDefined) {
app.provider.actorOf(Props(creator = actorFactory), parent.get, actorPath.name)
systemImpl.provider.actorOf(systemImpl, Props(creator = actorFactory), parent.get, actorPath.name)
} else {
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message)
}
@ -153,7 +152,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message)
}
sender ! Success(app.address)
sender ! Success(systemImpl.address)
} catch {
case error: Throwable //FIXME doesn't seem sensible
sender ! Failure(error)
@ -182,40 +181,47 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
// }
}
/*
* generate name for temporary actor refs
*/
private val tempNumber = new AtomicLong
def tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement())
def tempPath = remoteDaemon.path / tempName
// FIXME: handle real remote supervision
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
new LocalActorRef(systemImpl,
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
def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
new LocalActorRef(systemImpl,
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
def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
new LocalActorRef(systemImpl,
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
def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
new LocalActorRef(systemImpl,
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) {
@ -224,7 +230,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
}
private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = {
app.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
system.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
case Left(error) throw error
case Right(instance) instance.asInstanceOf[T]
}
@ -232,18 +238,21 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
}
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
val provider = remote.system.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
remote.system.deadLetters
lazy val recipient: ActorRef = remote.app.actorFor(input.getRecipient.getPath).getOrElse(remote.app.deadLetters)
lazy val recipient: ActorRef = remote.system.actorFor(input.getRecipient.getPath).getOrElse(remote.system.deadLetters)
lazy val payload: Either[Throwable, AnyRef] =
if (input.hasException) Left(parseException())
else Right(MessageSerializer.deserialize(remote.app, input.getMessage, classLoader))
else Right(MessageSerializer.deserialize(remote.system, input.getMessage, classLoader))
protected def parseException(): Throwable = {
val exception = input.getException
@ -256,7 +265,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
.newInstance(exception.getMessage).asInstanceOf[Throwable]
} catch {
case problem: Exception
remote.app.eventStream.publish(Logging.Error(problem, remote, problem.getMessage))
remote.system.eventStream.publish(Logging.Error(problem, remote, problem.getMessage))
CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
}
}
@ -266,7 +275,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
trait RemoteMarshallingOps {
def app: ActorSystem
def system: ActorSystem
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
val arp = AkkaRemoteProtocol.newBuilder
@ -284,7 +293,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 = system.asInstanceOf[ActorSystemImpl].provider.serialize(actor)
ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build
}
@ -297,7 +306,7 @@ trait RemoteMarshallingOps {
message match {
case Right(message)
messageBuilder.setMessage(MessageSerializer.serialize(app, message.asInstanceOf[AnyRef]))
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
case Left(exception)
messageBuilder.setException(ExceptionProtocol.newBuilder
.setClassname(exception.getClass.getName)

View file

@ -17,47 +17,63 @@ import akka.serialization.{ Serialization, Serializer, Compression }
import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress
import java.util.concurrent.ConcurrentHashMap
import com.google.protobuf.ByteString
import java.util.concurrent.atomic.AtomicBoolean
import akka.event.EventStream
/**
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
class RemoteActorRefProvider(
val settings: ActorSystem.Settings,
val rootPath: 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)
val remote = new Remote(app)
val local = new LocalActorRefProvider(settings, rootPath, eventStream, dispatcher, scheduler)
def deathWatch = local.deathWatch
def guardian = local.guardian
def systemGuardian = local.systemGuardian
def nodename = local.nodename
def tempName = local.tempName
@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(system: ActorSystemImpl) {
local.init(system)
remote = new Remote(system, nodename)
remoteDaemonConnectionManager = new RemoteConnectionManager(system, 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 = settings.ActorTimeout
def scheduler: Scheduler = local.scheduler
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(system, props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(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(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
if (systemService) local.actorOf(system, props, supervisor, path, systemService)
else {
val name = path.name
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
@ -76,13 +92,13 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
// case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass)
// }
def isReplicaNode: Boolean = remoteAddresses exists { _ == app.address }
def isReplicaNode: Boolean = remoteAddresses exists { _ == system.address }
//app.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(app.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
//system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
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(system, 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
@ -119,17 +135,17 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
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.system.provider, remote.server, remoteAddress, path, None))
}
val connectionManager = new RemoteConnectionManager(app, remote, connections)
val connectionManager = new RemoteConnectionManager(system, remote, connections)
connections.keys foreach { useActorOnNode(_, path.toString, props.creator) }
connections.keys foreach { useActorOnNode(system, _, path.toString, props.creator) }
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
}
case deploy local.actorOf(props, supervisor, name, systemService)
case deploy local.actorOf(system, props, supervisor, name, systemService)
}
} catch {
case e: Exception
@ -137,7 +153,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
throw e
}
// actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later
// actor foreach system.registry.register // only for ActorRegistry backward compat, will be removed later
newFuture completeWithResult actor
actors.replace(path.toString, newFuture, actor)
@ -151,9 +167,9 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
* Copied from LocalActorRefProvider...
*/
// FIXME: implement supervision
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
def actorOf(system: 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)
new RoutedActorRef(system, props, supervisor, name)
}
def actorFor(path: Iterable[String]): Option[ActorRef] = actors.get(ActorPath.join(path)) match {
@ -162,6 +178,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
case future: Future[_] Some(future.get.asInstanceOf[ActorRef])
}
// TODO remove me
val optimizeLocal = new AtomicBoolean(true)
def optimizeLocalScoped_?() = optimizeLocal.get
@ -177,22 +194,22 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
val remoteAddress = RemoteAddress(actor.hostname, actor.port)
if (optimizeLocalScoped_? && remoteAddress == app.address) {
if (optimizeLocalScoped_? && remoteAddress == rootPath.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 [{}]", rootPath.remoteAddress, actor.path, remoteAddress)
Some(RemoteActorRef(remote.system.provider, remote.server, remoteAddress, rootPath / 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(system: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) {
log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress)
val actorFactoryBytes =
app.serialization.serialize(actorFactory) match {
system.serialization.serialize(actorFactory) match {
case Left(error) throw error
case Right(bytes) if (remote.shouldCompressData) LZF.compress(bytes) else bytes
}
@ -242,8 +259,6 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = local.ask(message, recipient, within)
private[akka] def dummyAskSender = local.dummyAskSender
private[akka] def tempPath = local.tempPath
}
@ -254,6 +269,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
* @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,
@ -273,7 +289,7 @@ private[akka] case class RemoteActorRef private[akka] (
override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), remoteAddress, this, loader)
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = remote.app.provider.ask(message, this, timeout)
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = provider.ask(message, this, timeout)
def suspend(): Unit = ()
@ -289,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 startsWatching(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement

View file

@ -20,12 +20,12 @@ import java.util.concurrent.atomic.AtomicReference
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteConnectionManager(
app: ActorSystem,
system: ActorSystem,
remote: Remote,
initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef])
extends ConnectionManager {
val log = Logging(app, this)
val log = Logging(system, this)
// FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc.
case class State(version: Long, connections: Map[RemoteAddress, ActorRef])
@ -149,5 +149,5 @@ class RemoteConnectionManager(
}
private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) =
RemoteActorRef(remote.server, remoteAddress, actorPath, None)
RemoteActorRef(remote.system.provider, remote.server, remoteAddress, actorPath, None)
}

View file

@ -39,7 +39,7 @@ abstract class RemoteClient private[akka] (
val remoteSupport: NettyRemoteSupport,
val remoteAddress: RemoteAddress) {
val log = Logging(remoteSupport.app, this)
val log = Logging(remoteSupport.system, this)
val name = simpleName(this) + "@" + remoteAddress
@ -148,7 +148,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.system.rootPath.remoteAddress
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build)
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
}
@ -349,10 +350,10 @@ class ActiveRemoteClientHandler(
/**
* Provides the implementation of the Netty remote support
*/
class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
class NettyRemoteSupport(_system: ActorSystem) extends RemoteSupport(_system) with RemoteMarshallingOps {
val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
val serverSettings = new RemoteServerSettings(system.settings.config, system.settings.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(system.settings.config, system.settings.DefaultTimeUnit)
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val clientsLock = new ReentrantReadWriteLock
@ -447,7 +448,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@" + system.rootPath.remoteAddress
}
private val _isRunning = new Switch(false)
@ -479,9 +480,10 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
}
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
val log = Logging(remoteSupport.app, this)
val log = Logging(remoteSupport.system, this)
import remoteSupport.serverSettings._
import remoteSupport.app.address
val address = remoteSupport.system.rootPath.remoteAddress
val name = "NettyRemoteServer@" + address
@ -583,7 +585,7 @@ class RemoteServerHandler(
val applicationLoader: Option[ClassLoader],
val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler {
val log = Logging(remoteSupport.app, this)
val log = Logging(remoteSupport.system, this)
import remoteSupport.serverSettings._

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")
}

Some files were not shown because too many files have changed in this diff Show more