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 { public class JavaAPI {
private ActorSystem app = new ActorSystem(); private ActorSystem system = ActorSystem.create();
@Test void mustBeAbleToCreateActorRefFromClass() { @Test void mustBeAbleToCreateActorRefFromClass() {
ActorRef ref = app.actorOf(JavaAPITestActor.class); ActorRef ref = system.actorOf(JavaAPITestActor.class);
assertNotNull(ref); assertNotNull(ref);
} }
@Test void mustBeAbleToCreateActorRefFromFactory() { @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() { public Actor create() {
return new JavaAPITestActor(); return new JavaAPITestActor();
} }
@ -26,7 +26,7 @@ public class JavaAPI {
} }
@Test void mustAcceptSingleArgTell() { @Test void mustAcceptSingleArgTell() {
ActorRef ref = app.actorOf(JavaAPITestActor.class); ActorRef ref = system.actorOf(JavaAPITestActor.class);
ref.tell("hallo"); ref.tell("hallo");
ref.tell("hallo", ref); ref.tell("hallo", ref);
} }

View file

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

View file

@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec {
out.flush out.flush
out.close out.close
Serialization.app.withValue(app) { Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val readA = in.readObject 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] val a = actorOf[InnerActor]
import java.io._ import java.io._
@ -275,7 +275,7 @@ class ActorRefSpec extends AkkaSpec {
(intercept[java.lang.IllegalStateException] { (intercept[java.lang.IllegalStateException] {
in.readObject in.readObject
}).getMessage must be === "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." + }).getMessage must be === "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }" " Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }"
} }
"must throw exception on deserialize if not present in actor hierarchy (and remoting is not enabled)" in { "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 baos = new ByteArrayOutputStream(8192 * 32)
val out = new ObjectOutputStream(baos) 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.writeObject(serialized)
out.flush out.flush
out.close out.close
Serialization.app.withValue(app) { Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
(intercept[java.lang.IllegalStateException] { (intercept[java.lang.IllegalStateException] {
in.readObject in.readObject

View file

@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
} }
}, timeout = t)) }, timeout = t))
val defaultTimeout = app.AkkaConfig.ActorTimeout.duration val defaultTimeout = system.settings.ActorTimeout.duration
val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis val testTimeout = if (system.settings.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
"An Actor-based Future" must { "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 { "be able to parse 'akka.actor.cluster._' config elements" in {
// TODO: make it use its own special config? // TODO: make it use its own special config?
val config = app.config val config = system.settings.config
import config._ import config._
//akka.cluster //akka.cluster

View file

@ -14,7 +14,7 @@ class DeployerSpec extends AkkaSpec {
"A Deployer" must { "A Deployer" must {
"be able to parse 'akka.actor.deployment._' config elements" in { "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 be('defined)
deployment must equal(Some( deployment must equal(Some(

View file

@ -17,7 +17,7 @@ import akka.config.Configuration
object FSMActorSpec { object FSMActorSpec {
class Latches(implicit app: ActorSystem) { class Latches(implicit system: ActorSystem) {
val unlockedLatch = TestLatch() val unlockedLatch = TestLatch()
val lockedLatch = TestLatch() val lockedLatch = TestLatch()
val unhandledLatch = TestLatch() val unhandledLatch = TestLatch()
@ -167,12 +167,12 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
} }
}) })
filterException[Logging.EventHandlerException] { filterException[Logging.EventHandlerException] {
app.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
fsm ! "go" fsm ! "go"
expectMsgPF(1 second, hint = "Next state 2 does not exist") { expectMsgPF(1 second, hint = "Next state 2 does not exist") {
case Logging.Error(_, `fsm`, "Next state 2 does not exist") true 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 case StopEvent(r, _, _) testActor ! r
} }
}) })
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
fsm ! "go" fsm ! "go"
expectMsgPF(1 second, hint = "processing Event(go,null)") { 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, "setting timer 't'/1500 milliseconds: Shutdown"))
expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2")) expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2"))
fsm ! "stop" fsm ! "stop"
expectMsgPF(1 second, hint = "processing Event(stop,null)") { 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) expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal)
expectNoMsg(1 second) 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 { "A Finite State Machine" must {
"receive StateTimeout" in { "receive StateTimeout" taggedAs TimingTest in {
within(1 second) { within(1 second) {
within(500 millis, 1 second) { within(500 millis, 1 second) {
fsm ! TestStateTimeout 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) { within(1 second) {
fsm ! TestStateTimeout fsm ! TestStateTimeout
fsm ! Cancel 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) { within(500 millis) {
fsm ! TestStateTimeoutOverride fsm ! TestStateTimeoutOverride
expectNoMsg expectNoMsg
@ -56,8 +56,8 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
} }
} }
"receive single-shot timer" in { "receive single-shot timer" taggedAs TimingTest in {
within(1.5 seconds) { within(2 seconds) {
within(500 millis, 1 second) { within(500 millis, 1 second) {
fsm ! TestSingleTimer fsm ! TestSingleTimer
expectMsg(Tick) 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 fsm ! TestCancelTimer
within(500 millis) { within(500 millis) {
fsm ! Tick fsm ! Tick
@ -80,7 +80,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
expectMsg(1 second, Transition(fsm, TestCancelTimer, Initial)) 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 ! TestCancelStateTimerInNamedTimerMessage
fsm ! Tick fsm ! Tick
expectMsg(500 millis, 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 fsm ! TestRepeatedTimer
val seq = receiveWhile(2 seconds) { val seq = receiveWhile(2 seconds) {
case Tick Tick 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), 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)) { EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm, occurrences = 1)) {
fsm ! TestUnhandled fsm ! TestUnhandled

View file

@ -12,12 +12,12 @@ import akka.util.Duration
object ForwardActorSpec { object ForwardActorSpec {
val ExpectedMessage = "FOO" val ExpectedMessage = "FOO"
def createForwardingChain(app: ActorSystem): ActorRef = { def createForwardingChain(system: ActorSystem): ActorRef = {
val replier = app.actorOf(new Actor { val replier = system.actorOf(new Actor {
def receive = { case x sender ! x } def receive = { case x sender ! x }
}) })
def mkforwarder(forwardTo: ActorRef) = app.actorOf( def mkforwarder(forwardTo: ActorRef) = system.actorOf(
new Actor { new Actor {
def receive = { case x forwardTo forward x } 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 replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } })
val chain = createForwardingChain(app) val chain = createForwardingChain(system)
chain.tell(ExpectedMessage, replyTo) chain.tell(ExpectedMessage, replyTo)
latch.await(Duration(5, "s")) must be === true latch.await(Duration(5, "s")) must be === true
} }
"forward actor reference when invoking forward on bang bang" in { "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 chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage
} }
} }

View file

@ -13,14 +13,15 @@ class LocalActorRefProviderSpec extends AkkaSpec {
"An LocalActorRefProvider" must { "An LocalActorRefProvider" must {
"only create one instance of an actor with a specific address in a concurrent environment" in { "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) provider.isInstanceOf[LocalActorRefProvider] must be(true)
(0 until 100) foreach { i // 100 concurrent runs (0 until 100) foreach { i // 100 concurrent runs
val address = "new-actor" + i val address = "new-actor" + i
implicit val timeout = Timeout(5 seconds) 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 { "decorate a Receive" in {
new TestKit(appLogging) { new TestKit(appLogging) {
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val r: Actor.Receive = { val r: Actor.Receive = {
case null case null
} }
@ -66,8 +66,8 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"be added on Actor if requested" in { "be added on Actor if requested" in {
new TestKit(appLogging) with ImplicitSender { new TestKit(appLogging) with ImplicitSender {
ignoreMute(this) ignoreMute(this)
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
app.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this) { def receive = loggable(this) {
case _ sender ! "x" case _ sender ! "x"
@ -95,7 +95,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"not duplicate logging" in { "not duplicate logging" in {
new TestKit(appLogging) with ImplicitSender { new TestKit(appLogging) with ImplicitSender {
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this)(loggable(this) { def receive = loggable(this)(loggable(this) {
case _ sender ! "x" case _ sender ! "x"
@ -115,7 +115,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"log AutoReceiveMessages if requested" in { "log AutoReceiveMessages if requested" in {
new TestKit(appAuto) { new TestKit(appAuto) {
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = { def receive = {
case _ case _
@ -137,10 +137,10 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val s = ref.toString val s = ref.toString
s.contains("MainBusReaper") || s.contains("Supervisor") s.contains("MainBusReaper") || s.contains("Supervisor")
} }
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
app.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
within(3 seconds) { 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 supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
val supervisorSet = receiveWhile(messages = 2) { val supervisorSet = receiveWhile(messages = 2) {
@ -150,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
expectNoMsg(Duration.Zero) expectNoMsg(Duration.Zero)
assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)") 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) { val set = receiveWhile(messages = 2) {
case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" 1 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 { class RestartStrategySpec extends AkkaSpec {
override def atStartup { override def atStartup {
app.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
} }
object Ping object Ping

View file

@ -28,14 +28,14 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
def receive = { case Tick countDownLatch.countDown() } def receive = { case Tick countDownLatch.countDown() }
}) })
// run every 50 millisec // 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 // after max 1 second it should be executed at least the 3 times already
assert(countDownLatch.await(1, TimeUnit.SECONDS)) assert(countDownLatch.await(1, TimeUnit.SECONDS))
val countDownLatch2 = new CountDownLatch(3) 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 // after max 1 second it should be executed at least the 3 times already
assert(countDownLatch2.await(2, TimeUnit.SECONDS)) assert(countDownLatch2.await(2, TimeUnit.SECONDS))
@ -49,8 +49,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
}) })
// run every 50 millisec // run every 50 millisec
collectCancellable(app.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
collectCancellable(app.scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
// after 1 second the wait should fail // after 1 second the wait should fail
assert(countDownLatch.await(2, TimeUnit.SECONDS) == false) assert(countDownLatch.await(2, TimeUnit.SECONDS) == false)
@ -68,10 +68,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
// val actor = actorOf(new Actor { // val actor = actorOf(new Actor {
// def receive = { case Ping ticks.countDown } // 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))) // (1 to 1000).foreach(_ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS)))
// assert(ticks.await(10, TimeUnit.SECONDS)) // 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 (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() timeout.cancel()
} }
@ -114,10 +114,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
}) })
val actor = (supervisor ? props).as[ActorRef].get 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 // appx 2 pings before crash
EventFilter[Exception]("CRASH", occurrences = 1) intercept { 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)) 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 actor1 ! Kill
actor2 ! Kill actor2 ! Kill

View file

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

View file

@ -147,18 +147,18 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
newFooBar(Props().withTimeout(Timeout(d))) newFooBar(Props().withTimeout(Timeout(d)))
def newFooBar(props: Props): Foo = 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 = 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 { "TypedActors" must {
"be able to instantiate" in { "be able to instantiate" in {
val t = newFooBar val t = newFooBar
app.typedActor.isTypedActor(t) must be(true) system.typedActor.isTypedActor(t) must be(true)
mustStop(t) mustStop(t)
} }
@ -168,7 +168,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
} }
"not stop non-started ones" in { "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 { "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 { "be able to call toString" in {
val t = newFooBar val t = newFooBar
t.toString must be(app.typedActor.getActorRefFor(t).toString) t.toString must be(system.typedActor.getActorRefFor(t).toString)
mustStop(t) mustStop(t)
} }
@ -200,7 +200,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to call hashCode" in { "be able to call hashCode" in {
val t = newFooBar val t = newFooBar
t.hashCode must be(app.typedActor.getActorRefFor(t).hashCode) t.hashCode must be(system.typedActor.getActorRefFor(t).hashCode)
mustStop(t) mustStop(t)
} }
@ -295,7 +295,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
} }
"be able to support implementation only typed actors" in { "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 f = t.futurePigdog(200)
val f2 = t.futurePigdog(0) val f2 = t.futurePigdog(0)
f2.isCompleted must be(false) 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 { "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.stackable1 must be("foo")
t.stackable2 must be("bar") t.stackable2 must be("bar")
mustStop(t) mustStop(t)
@ -314,7 +314,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to use work-stealing dispatcher" in { "be able to use work-stealing dispatcher" in {
val props = Props( val props = Props(
timeout = Timeout(6600), timeout = Timeout(6600),
dispatcher = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher") dispatcher = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(60) .setCorePoolSize(60)
.setMaxPoolSize(60) .setMaxPoolSize(60)
@ -332,7 +332,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to serialize and deserialize invocations" in { "be able to serialize and deserialize invocations" in {
import java.io._ 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 baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos) 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)) 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] val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method) 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 { "be able to serialize and deserialize invocations' parameters" in {
import java.io._ import java.io._
val someFoo: Foo = new Bar 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 baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos) 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)) 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] val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method) mNew.method must be(m.method)

View file

@ -141,13 +141,13 @@ object ActorModelSpec {
} }
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)( 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 val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
try { try {
await(deadline)(stops == dispatcher.stops.get) await(deadline)(stops == dispatcher.stops.get)
} catch { } catch {
case e 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)) " required: stops=" + stops))
throw e throw e
} }
@ -173,7 +173,7 @@ object ActorModelSpec {
unregisters: Long = 0, unregisters: Long = 0,
msgsReceived: Long = 0, msgsReceived: Long = 0,
msgsProcessed: Long = 0, msgsProcessed: Long = 0,
restarts: Long = 0)(implicit app: ActorSystem) { restarts: Long = 0)(implicit system: ActorSystem) {
assertRef(actorRef, dispatcher)( assertRef(actorRef, dispatcher)(
suspensions, suspensions,
resumes, resumes,
@ -191,7 +191,7 @@ object ActorModelSpec {
unregisters: Long = statsFor(actorRef).unregisters.get(), unregisters: Long = statsFor(actorRef).unregisters.get(),
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(), msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(), msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: ActorSystem) { restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher)) val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val deadline = System.currentTimeMillis + 1000 val deadline = System.currentTimeMillis + 1000
try { try {
@ -204,7 +204,7 @@ object ActorModelSpec {
await(deadline)(stats.restarts.get() == restarts) await(deadline)(stats.restarts.get() == restarts)
} catch { } catch {
case e 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 + ",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts)) ",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
throw e throw e
@ -227,7 +227,7 @@ abstract class ActorModelSpec extends AkkaSpec {
import ActorModelSpec._ 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 newInterceptedDispatcher: MessageDispatcherInterceptor
protected def dispatcherType: String protected def dispatcherType: String
@ -310,7 +310,7 @@ abstract class ActorModelSpec extends AkkaSpec {
try { try {
f f
} catch { } 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._ import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, "foo", app.AkkaConfig.DispatcherThroughput, new Dispatcher(system.dispatcherFactory.prerequisites, "foo", system.settings.DispatcherThroughput,
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor] ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Dispatcher" def dispatcherType = "Dispatcher"
@ -444,14 +444,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, "foo", 1, // TODO check why 1 here? (came from old test) new BalancingDispatcher(system.dispatcherFactory.prerequisites, "foo", 1, // TODO check why 1 here? (came from old test)
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor] ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Balancing Dispatcher" def dispatcherType = "Balancing Dispatcher"
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = app.dispatcher override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = system.dispatcher
"A " + dispatcherType must { "A " + dispatcherType must {
"process messages in parallel" in { "process messages in parallel" in {

View file

@ -8,7 +8,7 @@ import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class BalancingDispatcherSpec extends AkkaSpec { 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() val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer()

View file

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

View file

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

View file

@ -27,14 +27,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
"support tell" in { "support tell" in {
var oneWay = new CountDownLatch(1) 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" val result = actor ! "OneWay"
assert(oneWay.await(1, TimeUnit.SECONDS)) assert(oneWay.await(1, TimeUnit.SECONDS))
actor.stop() actor.stop()
} }
"support ask/reply" in { "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] val result = (actor ? "Hello").as[String]
assert("World" === result.get) assert("World" === result.get)
actor.stop() 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 { "The default configuration file (i.e. akka-reference.conf)" must {
"contain all configuration properties for akka-actor that are used in code with their correct defaults" in { "contain all configuration properties for akka-actor that are used in code with their correct defaults" in {
val config = app.config val config = system.settings.config
import config._ import config._
getList("akka.boot") must equal(Nil) getList("akka.boot") must equal(Nil)

View file

@ -19,7 +19,7 @@ class Future2ActorSpec extends AkkaSpec {
} }
"support reply via sender" in { "support reply via sender" in {
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "do" Future(31) pipeTo context.sender case "do" Future(31) pipeTo context.sender
case "ex" Future(throw new AssertionError) 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 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) { def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
q must not be null q must not be null

View file

@ -18,12 +18,12 @@ class PriorityDispatcherSpec extends AkkaSpec {
testOrdering(BoundedPriorityMailbox(PriorityGenerator({ testOrdering(BoundedPriorityMailbox(PriorityGenerator({
case i: Int i //Reverse order case i: Int i //Reverse order
case 'Result Int.MaxValue case 'Result Int.MaxValue
}: Any Int), 1000, app.AkkaConfig.MailboxPushTimeout)) }: Any Int), 1000, system.settings.MailboxPushTimeout))
} }
} }
def testOrdering(mboxType: MailboxType) { 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 { val actor = actorOf(Props(new Actor {
var acc: List[Int] = Nil var acc: List[Int] = Nil

View file

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

View file

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

View file

@ -12,8 +12,9 @@ import java.util.concurrent.ThreadPoolExecutor.AbortPolicy
class TellThroughputPerformanceSpec extends PerformanceSpec { class TellThroughputPerformanceSpec extends PerformanceSpec {
import TellThroughputPerformanceSpec._ import TellThroughputPerformanceSpec._
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config new Dispatcher(app, name, 5, def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config
0, UnboundedMailbox(), config, 60000), ThreadPoolConfig(app)) new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
0, UnboundedMailbox(), config, 60000), ThreadPoolConfig())
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients) .setCorePoolSize(maxClients)
.build .build
@ -71,9 +72,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
val latch = new CountDownLatch(numberOfClients) val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until 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) 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 val start = System.nanoTime
clients.foreach(_ ! Run) clients.foreach(_ ! Run)
@ -150,4 +151,4 @@ object TellThroughputPerformanceSpec {
} }
} }

View file

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

View file

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

View file

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

View file

@ -11,8 +11,6 @@ import akka.actor.ActorSystem
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach { trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
def app: ActorSystem
def isBenchmark() = System.getProperty("benchmark") == "true" def isBenchmark() = System.getProperty("benchmark") == "true"
def minClients() = System.getProperty("benchmark.minClients", "1").toInt; def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
@ -29,7 +27,7 @@ trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
} }
val resultRepository = BenchResultRepository() 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 * 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 import scala.collection.immutable.TreeMap
class Report( class Report(
app: ActorSystem, system: ActorSystem,
resultRepository: BenchResultRepository, resultRepository: BenchResultRepository,
compareResultWith: Option[String] = None) { compareResultWith: Option[String] = None) {
private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean 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 dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
val legendTimeFormat = 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("Args:\n ").append(args)
sb.append("\n") sb.append("\n")
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion) sb.append("Akka version: ").append(system.settings.ConfigVersion)
sb.append("\n") sb.append("\n")
sb.append("Akka config:") sb.append("Akka config:")
for (key app.config.keys) { for (key system.settings.config.keys) {
sb.append("\n ").append(key).append("=").append(app.config(key)) sb.append("\n ").append(key).append("=").append(system.settings.config(key))
} }
sb.toString sb.toString

View file

@ -29,7 +29,7 @@ class TypedActorPoolSpec extends AkkaSpec {
import ActorPoolSpec._ import ActorPoolSpec._
"Actor Pool (2)" must { "Actor Pool (2)" must {
"support typed actors" in { "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 lowerBound = 1
def upperBound = 5 def upperBound = 5
def pressureThreshold = 1 def pressureThreshold = 1
@ -38,7 +38,7 @@ class TypedActorPoolSpec extends AkkaSpec {
def rampupRate = 0.1 def rampupRate = 0.1
def backoffRate = 0.50 def backoffRate = 0.50
def backoffThreshold = 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 def receive = _route
}, Props().withTimeout(10 seconds).withFaultHandler(faultHandler)) }, Props().withTimeout(10 seconds).withFaultHandler(faultHandler))
@ -47,7 +47,7 @@ class TypedActorPoolSpec extends AkkaSpec {
for ((i, r) results) for ((i, r) results)
r.get must equal(i * i) 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]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfiguredLocalRoutingSpec extends AkkaSpec { class ConfiguredLocalRoutingSpec extends AkkaSpec {
val deployer = system.asInstanceOf[ActorSystemImpl].provider.deployer
"round robin router" must { "round robin router" must {
"be able to shut down its instance" in { "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( Deploy(
path.toString, path.toString,
None, None,
@ -27,7 +29,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(5) val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" helloLatch.countDown() case "hello" helloLatch.countDown()
} }
@ -49,9 +51,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver messages in a round robin fashion" in { "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( Deploy(
path.toString, path.toString,
None, None,
@ -69,7 +71,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
replies = replies + (i -> 0) replies = replies + (i -> 0)
} }
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" sender ! id case "hit" sender ! id
@ -93,9 +95,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver a broadcast message using the !" in { "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( Deploy(
path.toString, path.toString,
None, None,
@ -106,7 +108,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(5) val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" helloLatch.countDown() case "hello" helloLatch.countDown()
} }
@ -127,9 +129,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"random router" must { "random router" must {
"be able to shut down its instance" in { "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( Deploy(
path.toString, path.toString,
None, None,
@ -139,7 +141,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val stopLatch = new CountDownLatch(7) val stopLatch = new CountDownLatch(7)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" {} case "hello" {}
} }
@ -160,9 +162,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver messages in a random fashion" in { "deliver messages in a random fashion" in {
val path = app / "random-1" val path = system / "random-1"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -180,7 +182,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
replies = replies + (i -> 0) replies = replies + (i -> 0)
} }
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" sender ! id case "hit" sender ! id
@ -204,9 +206,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver a broadcast message using the !" in { "deliver a broadcast message using the !" in {
val path = app / "random-2" val path = system / "random-2"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -217,7 +219,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(6) val helloLatch = new CountDownLatch(6)
val stopLatch = new CountDownLatch(6) val stopLatch = new CountDownLatch(6)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" helloLatch.countDown() case "hello" helloLatch.countDown()
} }

View file

@ -22,6 +22,8 @@ object RoutingSpec {
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class RoutingSpec extends AkkaSpec { class RoutingSpec extends AkkaSpec {
val impl = system.asInstanceOf[ActorSystemImpl]
import akka.routing.RoutingSpec._ import akka.routing.RoutingSpec._
"direct router" must { "direct router" must {
@ -29,7 +31,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1))) 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) 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 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 ! "hello"
routedActor ! "end" routedActor ! "end"
@ -66,7 +68,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1))) 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 ! Broadcast(1)
actor ! "end" actor ! "end"
@ -83,7 +85,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1))) 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) actor.isShutdown must be(false)
} }
@ -113,7 +115,7 @@ class RoutingSpec extends AkkaSpec {
//create the routed actor. //create the routed actor.
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(connections)) 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. //send messages to the actor.
for (i 0 until iterationCount) { 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 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(1)
actor ! Broadcast("end") actor ! Broadcast("end")
@ -175,7 +177,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1))) 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) } intercept[RoutingException] { actor ? Broadcast(1) }
@ -192,7 +194,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1))) 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) 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 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(1)
actor ! Broadcast("end") actor ! Broadcast("end")
@ -239,7 +241,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1))) 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 { try {
actor ? Broadcast(1) 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 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))) 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 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()) 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 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) (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 { "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 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) (actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
} }
"be started when constructed" in { "be started when constructed" in {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0)))) 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) actor.isShutdown must be(false)
} }
@ -324,7 +326,7 @@ class RoutingSpec extends AkkaSpec {
for (i 0 until connectionCount) { for (i 0 until connectionCount) {
counters = counters :+ new AtomicInteger() counters = counters :+ new AtomicInteger()
val connection = app.actorOf(new Actor { val connection = system.actorOf(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg) 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 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 (i 0 until iterationCount) {
for (k 0 until connectionCount) { for (k 0 until connectionCount) {
@ -357,7 +359,7 @@ class RoutingSpec extends AkkaSpec {
val doneLatch = new TestLatch(2) val doneLatch = new TestLatch(2)
val counter1 = new AtomicInteger val counter1 = new AtomicInteger
val connection1 = app.actorOf(new Actor { val connection1 = system.actorOf(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg) case msg: Int counter1.addAndGet(msg)
@ -365,7 +367,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val counter2 = new AtomicInteger val counter2 = new AtomicInteger
val connection2 = app.actorOf(new Actor { val connection2 = system.actorOf(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg) 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 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(1)
actor ! Broadcast("end") actor ! Broadcast("end")
@ -387,7 +389,7 @@ class RoutingSpec extends AkkaSpec {
case class Stop(id: Option[Int] = None) 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 = { def receive = {
case Stop(None) self.stop() case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) 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 akka.serialization.Serialization._
import scala.reflect._ import scala.reflect._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.actor.ActorSystem import akka.actor.{ ActorSystem, ActorSystemImpl }
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import akka.actor.DeadLetterActorRef import akka.actor.DeadLetterActorRef
@ -24,7 +24,8 @@ object SerializeSpec {
class SerializeSpec extends AkkaSpec { class SerializeSpec extends AkkaSpec {
import SerializeSpec._ import SerializeSpec._
import app.serialization._ val ser = system.serialization
import ser._
"Serialization" must { "Serialization" must {
@ -68,13 +69,13 @@ class SerializeSpec extends AkkaSpec {
"serialize DeadLetterActorRef" in { "serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream() val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf) val out = new ObjectOutputStream(outbuf)
val a = new ActorSystem() val a = ActorSystem()
out.writeObject(a.deadLetters) out.writeObject(a.deadLetters)
out.flush() out.flush()
out.close() out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
Serialization.app.withValue(a) { Serialization.system.withValue(a.asInstanceOf[ActorSystemImpl]) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef] val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) must be(true) (deadLetters eq a.deadLetters) must be(true)
} }

View file

@ -11,7 +11,7 @@ import org.junit.{ After, Test }
class CallingThreadDispatcherModelSpec extends ActorModelSpec { class CallingThreadDispatcherModelSpec extends ActorModelSpec {
import 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" def dispatcherType = "Calling Thread Dispatcher"
} }

View file

@ -150,11 +150,11 @@ object Timeout {
implicit def durationToTimeout(duration: Duration) = new Timeout(duration) implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
implicit def intToTimeout(timeout: Int) = new Timeout(timeout) implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
implicit def longToTimeout(timeout: Long) = new Timeout(timeout) implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
implicit def defaultTimeout(implicit app: ActorSystem) = app.AkkaConfig.ActorTimeout implicit def defaultTimeout(implicit system: ActorSystem) = system.settings.ActorTimeout
} }
trait ActorLogging { this: Actor trait ActorLogging { this: Actor
val log = akka.event.Logging(app.eventStream, context.self) val log = akka.event.Logging(system.eventStream, context.self)
} }
object Actor { object Actor {
@ -164,17 +164,17 @@ object Actor {
/** /**
* This decorator adds invocation logging to a Receive function. * This decorator adds invocation logging to a Receive function.
*/ */
class LoggingReceive(source: AnyRef, r: Receive)(implicit app: ActorSystem) extends Receive { class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive {
def isDefinedAt(o: Any) = { def isDefinedAt(o: Any) = {
val handled = r.isDefinedAt(o) 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 handled
} }
def apply(o: Any): Unit = r(o) def apply(o: Any): Unit = r(o)
} }
object LoggingReceive { 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 _: LoggingReceive r
case _ new LoggingReceive(source, r) case _ new LoggingReceive(source, r)
} }
@ -229,12 +229,12 @@ trait Actor {
c c
} }
implicit def app = context.app implicit def system = context.system
/** /**
* The default timeout, based on the config setting 'akka.actor.timeout' * 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 * 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 * This method does NOT modify the given Receive unless
* akka.actor.debug.receive is set within akka.conf. * 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. * 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 handleChildTerminated(child: ActorRef): Unit
def app: ActorSystem def system: ActorSystem
def parent: ActorRef def parent: ActorRef
} }
@ -63,7 +63,7 @@ private[akka] object ActorCell {
//vars don't need volatile since it's protected with the mailbox status //vars don't need volatile since it's protected with the mailbox status
//Make sure that they are not read/written outside of a message processing (systemInvoke/invoke) //Make sure that they are not read/written outside of a message processing (systemInvoke/invoke)
private[akka] class ActorCell( private[akka] class ActorCell(
val app: ActorSystem, val system: ActorSystemImpl,
val self: ActorRef with ScalaActorRef, val self: ActorRef with ScalaActorRef,
val props: Props, val props: Props,
val parent: ActorRef, val parent: ActorRef,
@ -72,11 +72,13 @@ private[akka] class ActorCell(
import ActorCell._ import ActorCell._
def systemImpl = system
protected final def guardian = self 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 var futureTimeout: Option[Cancellable] = None
@ -91,7 +93,7 @@ private[akka] class ActorCell(
var stopping = false var stopping = false
@inline @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 final def isShutdown: Boolean = mailbox.isClosed
@ -139,12 +141,12 @@ private[akka] class ActorCell(
} }
final def tell(message: Any, sender: ActorRef): Unit = 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 { final def sender: ActorRef = currentMessage match {
case null app.deadLetters case null system.deadLetters
case msg if msg.sender ne null msg.sender 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 //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 actor = created
created.preStart() created.preStart()
checkReceiveTimeout checkReceiveTimeout
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "started (" + actor + ")"))
} catch { } catch {
case e case e
try { 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 // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
} finally { } finally {
@ -186,7 +188,7 @@ private[akka] class ActorCell(
def recreate(cause: Throwable): Unit = try { def recreate(cause: Throwable): Unit = try {
val failedActor = actor 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() val freshActor = newActor()
if (failedActor ne null) { if (failedActor ne null) {
val c = currentMessage //One read only plz 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 actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
freshActor.postRestart(cause) 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? dispatcher.resume(this) //FIXME should this be moved down?
props.faultHandler.handleSupervisorRestarted(cause, self, children) props.faultHandler.handleSupervisorRestarted(cause, self, children)
} catch { } catch {
case e try { 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 // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
} finally { } finally {
@ -226,7 +228,7 @@ private[akka] class ActorCell(
val c = children val c = children
if (c.isEmpty) doTerminate() if (c.isEmpty) doTerminate()
else { 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() for (child c) child.stop()
stopping = true stopping = true
} }
@ -237,8 +239,8 @@ private[akka] class ActorCell(
if (!stats.contains(child)) { if (!stats.contains(child)) {
childrenRefs = childrenRefs.updated(child.name, child) childrenRefs = childrenRefs.updated(child.name, child)
childrenStats = childrenStats.updated(child, ChildRestartStats()) childrenStats = childrenStats.updated(child, ChildRestartStats())
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now supervising " + child))
} else app.eventStream.publish(Warning(self, "Already supervising " + child)) } else system.eventStream.publish(Warning(self, "Already supervising " + child))
} }
try { try {
@ -252,11 +254,11 @@ private[akka] class ActorCell(
case Create() create() case Create() create()
case Recreate(cause) recreate(cause) case Recreate(cause) recreate(cause)
case Link(subject) case Link(subject)
app.deathWatch.subscribe(self, subject) system.deathWatch.subscribe(self, subject)
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now monitoring " + subject))
case Unlink(subject) case Unlink(subject)
app.deathWatch.unsubscribe(self, subject) system.deathWatch.unsubscribe(self, subject)
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped monitoring " + subject))
case Suspend() suspend() case Suspend() suspend()
case Resume() resume() case Resume() resume()
case Terminate() terminate() case Terminate() terminate()
@ -265,7 +267,7 @@ private[akka] class ActorCell(
} }
} catch { } catch {
case e //Should we really catch everything here? 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? //TODO FIXME How should problems here be handled?
throw e throw e
} }
@ -284,7 +286,7 @@ private[akka] class ActorCell(
case msg case msg
if (stopping) { if (stopping) {
// receiving Terminated in response to stopping children is too common to generate noise // 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 { } else {
actor(msg) actor(msg)
} }
@ -292,7 +294,7 @@ private[akka] class ActorCell(
currentMessage = null // reset current message after successful invocation currentMessage = null // reset current message after successful invocation
} catch { } catch {
case e 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 // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
@ -312,7 +314,7 @@ private[akka] class ActorCell(
} }
} catch { } catch {
case e case e
app.eventStream.publish(Error(e, self, e.getMessage)) system.eventStream.publish(Error(e, self, e.getMessage))
throw e throw e
} }
} }
@ -330,11 +332,11 @@ private[akka] class ActorCell(
} }
def autoReceiveMessage(msg: Envelope) { 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 { if (stopping) msg.message match {
case ChildTerminated handleChildTerminated(sender) case ChildTerminated handleChildTerminated(sender)
case _ app.deadLetterMailbox.enqueue(self, msg) case _ system.deadLetterMailbox.enqueue(self, msg)
} }
else msg.message match { else msg.message match {
case HotSwap(code, discardOld) become(code(self), discardOld) case HotSwap(code, discardOld) become(code(self), discardOld)
@ -347,8 +349,8 @@ private[akka] class ActorCell(
} }
private def doTerminate() { private def doTerminate() {
if (!app.provider.evict(self.path.toString)) if (!system.provider.evict(self.path.toString))
app.eventStream.publish(Warning(self, "evict of " + self.path.toString + " failed")) system.eventStream.publish(Warning(self, "evict of " + self.path.toString + " failed"))
dispatcher.detach(this) dispatcher.detach(this)
@ -358,8 +360,8 @@ private[akka] class ActorCell(
} finally { } finally {
try { try {
parent.tell(ChildTerminated, self) parent.tell(ChildTerminated, self)
app.deathWatch.publish(Terminated(self)) system.deathWatch.publish(Terminated(self))
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped"))
} finally { } finally {
currentMessage = null currentMessage = null
clearActorFields() clearActorFields()
@ -369,7 +371,7 @@ private[akka] class ActorCell(
final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenStats.get(child) match { 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 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 = { final def handleChildTerminated(child: ActorRef): Unit = {
@ -387,7 +389,7 @@ private[akka] class ActorCell(
val recvtimeout = receiveTimeout val recvtimeout = receiveTimeout
if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) {
//Only reschedule if desired and there are currently no more messages to be processed //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 package akka.actor
import akka.remote.RemoteAddress
object ActorPath { object ActorPath {
final val separator = "/" final val separator = "/"
@ -12,14 +14,14 @@ object ActorPath {
/** /**
* Create an actor path from a string. * Create an actor path from a string.
*/ */
def apply(app: ActorSystem, path: String): ActorPath = def apply(system: ActorSystem, path: String): ActorPath =
apply(app, split(path)) apply(system, split(path))
/** /**
* Create an actor path from an iterable. * Create an actor path from an iterable.
*/ */
def apply(app: ActorSystem, path: Iterable[String]): ActorPath = def apply(system: ActorSystem, path: Iterable[String]): ActorPath =
path.foldLeft(app.root)(_ / _) path.foldLeft(system.rootPath)(_ / _)
/** /**
* Split a string path into an iterable. * Split a string path into an iterable.
@ -58,9 +60,9 @@ object ActorPath {
*/ */
trait 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. * The name of the actor that this path refers to.
@ -78,9 +80,9 @@ trait ActorPath {
def /(child: String): 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. * String representation of this path. Different from toString for root path.
@ -98,15 +100,13 @@ trait ActorPath {
def isRoot: Boolean def isRoot: Boolean
} }
class RootActorPath(val app: ActorSystem) extends ActorPath { class RootActorPath(val remoteAddress: RemoteAddress) extends ActorPath {
def name: String = "/" def name: String = "/"
def parent: ActorPath = this def parent: ActorPath = this
def /(child: String): ActorPath = new ChildActorPath(app, this, child) def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
def ref: Option[ActorRef] = app.actorFor(path)
def string: String = "" def string: String = ""
@ -117,11 +117,9 @@ class RootActorPath(val app: ActorSystem) extends ActorPath {
override def toString = ActorPath.separator 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 /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
def ref: Option[ActorRef] = app.actorFor(path)
def string: String = parent.string + ActorPath.separator + name def string: String = parent.string + ActorPath.separator + name

View file

@ -12,6 +12,8 @@ import akka.serialization.Serialization
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.remote.RemoteAddress import akka.remote.RemoteAddress
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.event.EventStream
import akka.event.DeathWatch
/** /**
* ActorRef is an immutable and serializable handle to an Actor. * 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class LocalActorRef private[akka] ( class LocalActorRef private[akka] (
app: ActorSystem, system: ActorSystemImpl,
_props: Props, _props: Props,
_supervisor: ActorRef, _supervisor: ActorRef,
val path: ActorPath, val path: ActorPath,
@ -171,10 +173,19 @@ class LocalActorRef private[akka] (
def name = path.name 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 @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() actorCell.start()
/** /**
@ -295,17 +306,17 @@ trait ScalaActorRef { ref: ActorRef ⇒
*/ */
case class SerializedActorRef(hostname: String, port: Int, path: String) { 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: 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 def this(remoteAddress: InetSocketAddress, path: String) = this(remoteAddress.getAddress.getHostAddress, remoteAddress.getPort, path) //TODO FIXME REMOVE
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
def readResolve(): AnyRef = { def readResolve(): AnyRef = {
if (app.value eq null) throw new IllegalStateException( if (system.value eq null) throw new IllegalStateException(
"Trying to deserialize a serialized ActorRef without an ActorSystem in scope." + "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) { ... }")
app.value.provider.deserialize(this) match { system.value.provider.deserialize(this) match {
case Some(actor) actor case Some(actor) actor
case None throw new IllegalStateException("Could not deserialize ActorRef") case None throw new IllegalStateException("Could not deserialize ActorRef")
} }
@ -344,31 +355,35 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
object DeadLetterActorRef { object DeadLetterActorRef {
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
@throws(classOf[java.io.ObjectStreamException]) @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 val serialized = new SerializedDeadLetterActorRef
} }
class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef { class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) extends MinimalActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) @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" override val name: String = "dead-letter"
// FIXME (actor path): put this under the sys guardian supervisor def address: String = path.toString
val path: ActorPath = app.root / "sys" / name
def address: String = app.address + path.toString
override def isShutdown(): Boolean = true override def isShutdown(): Boolean = true
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match { override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
case d: DeadLetter app.eventStream.publish(d) case d: DeadLetter eventStream.publish(d)
case _ app.eventStream.publish(DeadLetter(message, sender, this)) case _ eventStream.publish(DeadLetter(message, sender, this))
} }
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { 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 brokenPromise
} }
@ -376,16 +391,15 @@ class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized 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) final val result = new DefaultPromise[Any](timeout)(dispatcher)
// FIXME (actor path): put this under the tmp guardian supervisor override def name = path.name
val path: ActorPath = app.root / "tmp" / 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 onComplete callback
result onTimeout 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")) override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped"))
@throws(classOf[java.io.ObjectStreamException]) @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 package akka.actor
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.{ ConcurrentHashMap, TimeUnit }
import java.util.concurrent.{ TimeUnit, Executors }
import scala.annotation.tailrec import scala.annotation.tailrec
import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer } import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer }
import akka.actor.Timeout.intToTimeout import akka.actor.Timeout.intToTimeout
import akka.config.ConfigurationException import akka.config.ConfigurationException
import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise } import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise, Dispatcher, Mailbox, Envelope }
import akka.event.{ Logging, DeathWatch, ActorClassification } import akka.event.{ Logging, DeathWatch, ActorClassification, EventStream }
import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter } import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter }
import akka.util.Helpers import akka.util.Helpers
import akka.AkkaException import akka.AkkaException
import com.eaio.uuid.UUID
/** /**
* Interface for all ActorRef providers to implement. * Interface for all ActorRef providers to implement.
*/ */
trait ActorRefProvider { 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] def actorFor(path: Iterable[String]): Option[ActorRef]
/** def guardian: ActorRef
* What deployer will be used to resolve deployment configuration?
*/ 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 deployer: Deployer
private[akka] def scheduler: Scheduler 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 evict(path: String): Boolean
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef]
@ -48,15 +66,16 @@ trait ActorRefProvider {
private[akka] def createDeathWatch(): DeathWatch 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 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 terminationFuture: Future[ActorSystem.ExitStatus]
private[akka] def dummyAskSender: ActorRef
private[akka] def tempPath: String
} }
/** /**
@ -64,9 +83,11 @@ trait ActorRefProvider {
*/ */
trait ActorRefFactory { 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. * Father of all children created by this interface.
@ -80,7 +101,7 @@ trait ActorRefFactory {
Helpers.base64(l) 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 * 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 = { def actorOf(props: Props, name: String): ActorRef = {
if (name == null || name == "" || name.startsWith("$")) if (name == null || name == "" || name.startsWith("$"))
throw new ActorInitializationException("actor name must not be null, empty or start with $") 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]])) 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 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: String): Option[ActorRef] = actorFor(ActorPath.split(path))
def actorFor(path: Iterable[String]): Option[ActorRef] = provider.actorFor(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. * 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) // FIXME remove/replave (clustering shall not leak into akka-actor)
val nodename: String = System.getProperty("akka.cluster.nodename") match {
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher) case null | "" new UUID().toString
case value value
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
} }
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 * Top-level anchor for the supervision hierarchy of this actor system. Will
* receive only Supervise/ChildTerminated system messages or Failure message. * receive only Supervise/ChildTerminated system messages or Failure message.
@ -136,12 +176,12 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
@volatile @volatile
var stopped = false 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 // 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 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 private class Guardian extends Actor {
// currently still used for tmp actors (e.g. ask actor refs) def receive = {
private val actors = new ConcurrentHashMap[String, AnyRef] 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 @tailrec
private def findInTree(start: Option[ActorRef], path: Iterable[String]): Option[ActorRef] = { 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 evict(path: String): Boolean = actors.remove(path) ne null
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(props, supervisor, supervisor.path / name, systemService) 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 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 { actors.putIfAbsent(path.toString, newFuture) match {
case null case null
@ -207,7 +292,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
// create a local actor // create a local actor
case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, DeploymentConfig.LocalScope)) 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 // create a routed actor ref
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.LocalScope)) 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.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()( 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.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet") case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages 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 connections: Iterable[ActorRef] = (1 to nrOfInstances.factor) map { i
val routedPath = path.parent / (path.name + ":" + 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) 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. * 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: this needs to take supervision into account!
//FIXME clustering should be implemented by cluster actor ref provider //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 // val localOnly = props.localOnly
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props) // if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
// else new RoutedActorRef(props, address) // 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 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 createDeathWatch(): DeathWatch = new LocalDeathWatch
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = { private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = {
import akka.dispatch.DefaultPromise 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 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 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 assert(actors.putIfAbsent(a.path.toString, a) eq null) //If this fails, we're in deep trouble
recipient.tell(message, a) recipient.tell(message, a)
a.result 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 { class LocalDeathWatch extends DeathWatch with ActorClassification {

View file

@ -6,19 +6,19 @@ package akka.actor
import akka.config._ import akka.config._
import akka.actor._ import akka.actor._
import akka.event._ import akka.event._
import akka.dispatch._
import akka.util.duration._ import akka.util.duration._
import java.net.InetAddress import java.net.InetAddress
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.dispatch.{ Dispatchers, Future, Mailbox, Envelope, SystemMessage }
import akka.util.Duration import akka.util.Duration
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.serialization.Serialization import akka.serialization.Serialization
import akka.remote.RemoteAddress import akka.remote.RemoteAddress
import org.jboss.netty.akka.util.HashedWheelTimer
import java.util.concurrent.{ Executors, TimeUnit }
object ActorSystem { object ActorSystem {
type AkkaConfig = a.AkkaConfig.type forSome { val a: ActorSystem }
val Version = "2.0-SNAPSHOT" val Version = "2.0-SNAPSHOT"
val envHome = System.getenv("AKKA_HOME") match { val envHome = System.getenv("AKKA_HOME") match {
@ -61,26 +61,20 @@ object ActorSystem {
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
def apply(name: String, config: Configuration) = new 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 sealed trait ExitStatus
case object Stopped extends ExitStatus case object Stopped extends ExitStatus
case class Failed(cause: Throwable) extends ExitStatus case class Failed(cause: Throwable) extends ExitStatus
} class Settings(val config: Configuration) {
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 {
import config._ import config._
val ConfigVersion = getString("akka.version", Version) 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 FailureDetectorThreshold: Int = getInt("akka.remote.failure-detector.threshold", 8)
val FailureDetectorMaxSampleSize: Int = getInt("akka.remote.failure-detector.max-sample-size", 1000) val FailureDetectorMaxSampleSize: Int = getInt("akka.remote.failure-detector.max-sample-size", 1000)
if (ConfigVersion != Version)
throw new ConfigurationException("Akka JAR version [" + Version +
"] does not match the provided config version [" + ConfigVersion + "]")
} }
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true) }
import AkkaConfig._ abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
import ActorSystem._
if (ConfigVersion != Version) def name: String
throw new ConfigurationException("Akka JAR version [" + Version + def settings: Settings
"] does not match the provided config version [" + ConfigVersion + "]") def nodename: String
/**
* Construct a path below the application guardian.
*/
def /(name: String): ActorPath
def rootPath: ActorPath
val startTime = System.currentTimeMillis val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000 def uptime = (System.currentTimeMillis - startTime) / 1000
val nodename: String = System.getProperty("akka.cluster.nodename") match { def eventStream: EventStream
case null | "" new UUID().toString def log: LoggingAdapter
case value value
} 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 { val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostAddress case null | "" InetAddress.getLocalHost.getHostAddress
case value value case value value
}, System.getProperty("akka.remote.port") match { }, System.getProperty("akka.remote.port") match {
case null | "" AkkaConfig.RemoteServerPort case null | "" settings.RemoteServerPort
case value value.toInt case value value.toInt
}) })
// this provides basic logging (to stdout) until .start() is called below // this provides basic logging (to stdout) until .start() is called below
val eventStream = new EventStream(DebugEventStream) val eventStream = new EventStream(DebugEventStream)
eventStream.startStdoutLogger(AkkaConfig) eventStream.startStdoutLogger(settings)
val log = new BusLogging(eventStream, this) val log = new BusLogging(eventStream, this) // this used only for .getClass in tagging messages
// 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)
/** /**
* The root actor path for this application. * 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 deadLetters = new DeadLetterActorRef(eventStream, rootPath / "nul")
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 deadLetterMailbox = new Mailbox(null) { val deadLetterMailbox = new Mailbox(null) {
becomeClosed() becomeClosed()
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } 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 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 // TODO correctly pull its config from the config
deathWatch.subscribe(systemGuardian, guardian) val dispatcherFactory = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler))
deathWatch.subscribe(rootGuardian, systemGuardian) implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
// this starts the reaper actor and the user-configured logging subscribers, which are also actors deadLetters.init(dispatcher)
eventStream.start(this)
eventStream.startDefaultLoggers(this, AkkaConfig)
// TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor val provider: ActorRefProvider = {
val deployer = new Deployer(this) 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
// TODO think about memory consistency effects when doing funky stuff inside constructor ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
val typedActor = new TypedActor(this) case Left(e) throw e
case Right(p) p
}
}
// TODO think about memory consistency effects when doing funky stuff inside constructor def terminationFuture: Future[ExitStatus] = provider.terminationFuture
val serialization = new Serialization(this) 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
/**
* Create an actor path under the application supervisor (/app).
*/
def /(actorName: String): ActorPath = guardian.path / actorName 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)
this
}
def registerOnTermination(code: Unit) { terminationFuture onComplete (_ code) }
def registerOnTermination(code: Runnable) { terminationFuture onComplete (_ code.run) }
// TODO shutdown all that other stuff, whatever that may be // TODO shutdown all that other stuff, whatever that may be
def stop() { def stop() {
guardian.stop() guardian.stop()
} }
terminationFuture.onComplete(_ dispatcher.shutdown())
} }

View file

@ -14,14 +14,14 @@ import akka.util.Bootable
*/ */
trait BootableActorLoaderService extends 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() lazy val applicationLoader = createApplicationClassLoader()
protected def createApplicationClassLoader(): Option[ClassLoader] = Some({ protected def createApplicationClassLoader(): Option[ClassLoader] = Some({
if (app.AkkaConfig.Home.isDefined) { if (system.settings.Home.isDefined) {
val DEPLOY = app.AkkaConfig.Home.get + "/deploy" val DEPLOY = system.settings.Home.get + "/deploy"
val DEPLOY_DIR = new File(DEPLOY) val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) { if (!DEPLOY_DIR.exists) {
System.exit(-1) System.exit(-1)
@ -59,11 +59,11 @@ trait BootableActorLoaderService extends Bootable {
super.onUnload() super.onUnload()
// FIXME shutdown all actors // FIXME shutdown all actors
// app.registry.local.shutdownAll // system.registry.local.shutdownAll
} }
} }
/** /**
* Java API for the default JAX-RS/Mist Initializer * 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 package akka.actor
import collection.immutable.Seq import collection.immutable.Seq
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import akka.event.Logging import akka.event.Logging
import akka.actor.DeploymentConfig._ import akka.actor.DeploymentConfig._
import akka.AkkaException import akka.AkkaException
@ -15,6 +13,7 @@ import akka.config.{ Configuration, ConfigurationException }
import akka.util.Duration import akka.util.Duration
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.remote.RemoteAddress import akka.remote.RemoteAddress
import akka.event.EventStream
trait ActorDeployer { trait ActorDeployer {
private[akka] def init(deployments: Seq[Deploy]): Unit 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> * @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 deploymentConfig = new DeploymentConfig(nodename)
val log = Logging(app.eventStream, this) val log = Logging(eventStream, this)
val instance: ActorDeployer = { val instance: ActorDeployer = {
val deployer = new LocalDeployer() val deployer = new LocalDeployer()
@ -86,7 +85,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer {
private[akka] def pathsInConfig: List[String] = { private[akka] def pathsInConfig: List[String] = {
val deploymentPath = "akka.actor.deployment" val deploymentPath = "akka.actor.deployment"
app.config.getSection(deploymentPath) match { settings.config.getSection(deploymentPath) match {
case None Nil case None Nil
case Some(pathConfig) case Some(pathConfig)
pathConfig.map.keySet pathConfig.map.keySet
@ -98,7 +97,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer {
/** /**
* Lookup deployment in 'akka.conf' configuration file. * 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 } 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class DeploymentConfig(val app: ActorSystem) { class DeploymentConfig(val nodename: String) {
import DeploymentConfig._ 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 { def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match {
case Deploy(_, _, _, _, ClusterScope(_, replicationScheme)) Some(replicationScheme) case Deploy(_, _, _, _, ClusterScope(_, replicationScheme)) Some(replicationScheme)

View file

@ -28,14 +28,14 @@ object FSM {
case object StateTimeout case object StateTimeout
case class TimeoutMarker(generation: Long) case class TimeoutMarker(generation: Long)
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: ActorSystem) { case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) {
private var ref: Option[Cancellable] = _ private var ref: Option[Cancellable] = _
def schedule(actor: ActorRef, timeout: Duration) { def schedule(actor: ActorRef, timeout: Duration) {
if (repeat) { 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 { } 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 Timeout = Option[Duration]
type TransitionHandler = PartialFunction[(S, S), Unit] 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) { if (timeout.isDefined) {
val t = timeout.get val t = timeout.get
if (t.finite_? && t.length >= 0) { 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 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 events = new Array[Event](logDepth)
private val states = new Array[AnyRef](logDepth) private val states = new Array[AnyRef](logDepth)

View file

@ -253,7 +253,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor {
var worker: IOWorker = _ var worker: IOWorker = _
override def preStart { override def preStart {
worker = new IOWorker(app, self, bufferSize) worker = new IOWorker(system, self, bufferSize)
worker.start() worker.start()
} }
@ -290,7 +290,7 @@ private[akka] object IOWorker {
case object Shutdown extends Request 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 SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT }
import IOWorker._ 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 * This class represents a Method call, and has a reference to the Method to be called and the parameters to supply
* It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized * It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized
*/ */
case class MethodCall(app: 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 isOneWay = method.getReturnType == java.lang.Void.TYPE
def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType) 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 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 if ps.length == 0 SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array[Serializer.Identifier](), Array[Array[Byte]]())
case ps 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) val serializedParameters: Array[Array[Byte]] = Array.ofDim[Array[Byte]](serializers.length)
for (i 0 until serializers.length) for (i 0 until serializers.length)
serializedParameters(i) = serializers(i) toBinary parameters(i) //Mutable for the sake of sanity 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 implement writeObject and readObject to serialize
//TODO Possible optimization is to special encode the parameter-types to conserve space //TODO Possible optimization is to special encode the parameter-types to conserve space
private def readResolve(): AnyRef = { private def readResolve(): AnyRef = {
val app = akka.serialization.Serialization.app.value val system = akka.serialization.Serialization.system.value
if (app eq null) throw new IllegalStateException( if (system eq null) throw new IllegalStateException(
"Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." + "Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }") " Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }")
MethodCall(app, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { MethodCall(system.serialization, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match {
case null null case null null
case a if a.length == 0 Array[AnyRef]() case a if a.length == 0 Array[AnyRef]()
case a case a
val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity
for (i 0 until a.length) { 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 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 { def system = appReference.get match {
case null throw new IllegalStateException("Calling TypedActor.app outside of a TypedActor implementation method!") case null throw new IllegalStateException("Calling TypedActor.system outside of a TypedActor implementation method!")
case some some case some some
} }
/** /**
* Returns the default dispatcher (for a TypedActor) when inside a method call in a TypedActor. * 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. * 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 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) * 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 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 //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
val actorVar = new AtomVar[ActorRef](null) val actorVar = new AtomVar[ActorRef](null)
val timeout = props.timeout match { val timeout = props.timeout match {
case Props.`defaultTimeout` app.AkkaConfig.ActorTimeout case Props.`defaultTimeout` settings.ActorTimeout
case x x case x x
} }
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T] val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T]
@ -330,7 +330,7 @@ class TypedActor(val app: ActorSystem) {
def receive = { def receive = {
case m: MethodCall case m: MethodCall
TypedActor.selfReference set proxyVar.get TypedActor.selfReference set proxyVar.get
TypedActor.appReference set app TypedActor.appReference set system
try { try {
if (m.isOneWay) m(me) if (m.isOneWay) m(me)
else { 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 "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 "hashCode" actor.hashCode.asInstanceOf[AnyRef]
case _ 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.isOneWay actor ! m; null //Null return value
case m if m.returnsFuture_? actor.?(m, timeout) case m if m.returnsFuture_? actor.?(m, timeout)
case m if m.returnsJOption_? || m.returnsOption_? 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 actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef] 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 type Uuid = com.eaio.uuid.UUID
def newUuid(): Uuid = new Uuid() def newUuid(): Uuid = new Uuid()

View file

@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) {
*/ */
object NodeAddress { object NodeAddress {
def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName) def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName)
def apply(app: 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 { def unapply(other: Any) = other match {
case address: NodeAddress Some((address.clusterName, address.nodeName)) 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 * 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 { class ResourceImporter(classLoader: ClassLoader) extends Importer {
def importFile(filename: String): String = { def importFile(filename: String): String = {

View file

@ -14,6 +14,8 @@ import akka.actor._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import locks.ReentrantLock import locks.ReentrantLock
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.event.EventStream
import akka.actor.ActorSystem.Settings
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @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 Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsWatching
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsWatching 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() { def run() {
try { try {
function() function()
} catch { } catch {
case e app.eventStream.publish(Error(e, this, e.getMessage)) case e eventStream.publish(Error(e, this, e.getMessage))
} finally { } finally {
cleanup() cleanup()
} }
@ -79,26 +81,23 @@ object MessageDispatcher {
val SCHEDULED = 1 val SCHEDULED = 1
val RESCHEDULED = 2 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> * @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 MessageDispatcher._
import AbstractMessageDispatcher.{ inhabitantsUpdater, shutdownScheduleUpdater } import AbstractMessageDispatcher.{ inhabitantsUpdater, shutdownScheduleUpdater }
import prerequisites._
/** /**
* Creates and returns a mailbox for the given actor. * Creates and returns a mailbox for the given actor.
*/ */
protected[akka] def createMailbox(actor: ActorCell): Mailbox 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. * Name of this dispatcher.
*/ */
@ -119,7 +118,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
} }
protected[akka] final def dispatchTask(block: () Unit) { protected[akka] final def dispatchTask(block: () Unit) {
val invocation = TaskInvocation(app, block, taskCleanup) val invocation = TaskInvocation(eventStream, block, taskCleanup)
inhabitantsUpdater.incrementAndGet(this) inhabitantsUpdater.incrementAndGet(this)
try { try {
executeTask(invocation) executeTask(invocation)
@ -136,7 +135,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
shutdownScheduleUpdater.get(this) match { shutdownScheduleUpdater.get(this) match {
case UNSCHEDULED case UNSCHEDULED
if (shutdownScheduleUpdater.compareAndSet(this, UNSCHEDULED, SCHEDULED)) { if (shutdownScheduleUpdater.compareAndSet(this, UNSCHEDULED, SCHEDULED)) {
app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
() ()
} else ifSensibleToDoSoThenScheduleShutdown() } else ifSensibleToDoSoThenScheduleShutdown()
case SCHEDULED case SCHEDULED
@ -211,7 +210,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
} }
case RESCHEDULED case RESCHEDULED
if (shutdownScheduleUpdater.compareAndSet(MessageDispatcher.this, RESCHEDULED, SCHEDULED)) if (shutdownScheduleUpdater.compareAndSet(MessageDispatcher.this, RESCHEDULED, SCHEDULED))
app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
else run() 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 * 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 * 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 = { def mailboxType(config: Configuration, settings: Settings): MailboxType = {
val capacity = config.getInt("mailbox-capacity", app.AkkaConfig.MailboxCapacity) val capacity = config.getInt("mailbox-capacity", settings.MailboxCapacity)
if (capacity < 1) UnboundedMailbox() if (capacity < 1) UnboundedMailbox()
else { else {
val duration = Duration( val duration = Duration(
config.getInt("mailbox-push-timeout-time", app.AkkaConfig.MailboxPushTimeout.toMillis.toInt), config.getInt("mailbox-push-timeout-time", settings.MailboxPushTimeout.toMillis.toInt),
app.AkkaConfig.DefaultTimeUnit) settings.DefaultTimeUnit)
BoundedMailbox(capacity, duration) BoundedMailbox(capacity, duration)
} }
} }
def configureThreadPool(config: Configuration, createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = { def configureThreadPool(config: Configuration,
settings: Settings,
createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
import ThreadPoolConfigDispatcherBuilder.conf_? import ThreadPoolConfigDispatcherBuilder.conf_?
//Apply the following options to the config if they are present in the config //Apply the following options to the config if they are present in the config
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(app)).configure( ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()).configure(
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, app.AkkaConfig.DefaultTimeUnit))), 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 "core-pool-size-factor")(factor _.setCorePoolSizeFromFactor(factor)),
conf_?(config getDouble "max-pool-size-factor")(factor _.setMaxPoolSizeFromFactor(factor)), conf_?(config getDouble "max-pool-size-factor")(factor _.setMaxPoolSizeFromFactor(factor)),
conf_?(config getBool "allow-core-timeout")(allow _.setAllowCoreThreadTimeout(allow)), 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 java.util.{ Comparator, Queue }
import annotation.tailrec import annotation.tailrec
import akka.actor.ActorSystem 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 * 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 * @author Viktor Klang
*/ */
class BalancingDispatcher( class BalancingDispatcher(
_app: ActorSystem, _prerequisites: DispatcherPrerequisites,
_name: String, _name: String,
throughput: Int, throughput: Int,
throughputDeadlineTime: Int, throughputDeadlineTime: Int,
mailboxType: MailboxType, mailboxType: MailboxType,
config: ThreadPoolConfig, config: ThreadPoolConfig,
_timeoutMs: Long) _timeoutMs: Long)
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) { extends Dispatcher(_prerequisites, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
import app.deadLetterMailbox
private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator) 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 // message must be virgin before being able to systemEnqueue again
val next = message.next val next = message.next
message.next = null message.next = null
deadLetterMailbox.systemEnqueue(actor.self, message) prerequisites.deadLetterMailbox.systemEnqueue(actor.self, message)
message = next message = next
} }
} }

View file

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

View file

@ -10,6 +10,20 @@ import akka.util.{ Duration, ReflectiveAccess }
import akka.config.Configuration import akka.config.Configuration
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.actor.ActorSystem 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. * Scala API. Dispatcher factory.
@ -41,15 +55,16 @@ import akka.actor.ActorSystem
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Dispatchers(val app: ActorSystem) { class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: DispatcherPrerequisites) {
val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
val ThroughputDeadlineTimeMillis = settings.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType = val MailboxType: MailboxType =
if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox() if (settings.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(app.AkkaConfig.MailboxCapacity, app.AkkaConfig.MailboxPushTimeout) else BoundedMailbox(settings.MailboxCapacity, settings.MailboxPushTimeout)
val DispatcherShutdownMillis = app.AkkaConfig.DispatcherDefaultShutdown.toMillis val DispatcherShutdownMillis = settings.DispatcherDefaultShutdown.toMillis
lazy val defaultGlobalDispatcher = 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. * 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. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(actor: LocalActorRef) = actor match { def newPinnedDispatcher(actor: LocalActorRef) = actor match {
case null new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis) case null new PinnedDispatcher(prerequisites, null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.address, 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. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match { def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
case null new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis) case null new PinnedDispatcher(prerequisites, null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.address, 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. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(name: String, mailboxType: MailboxType) = 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. * 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. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(name: String) = 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. * 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. * Has a fluent builder interface for configuring its semantics.
*/ */
def newDispatcher(name: String) = def newDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(app, name, app.AkkaConfig.DispatcherThroughput, ThreadPoolConfigDispatcherBuilder(config new Dispatcher(prerequisites, name, settings.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/** /**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. * 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) = def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config 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. * 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) = def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config 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. * 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. * Has a fluent builder interface for configuring its semantics.
*/ */
def newBalancingDispatcher(name: String) = def newBalancingDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(app, name, app.AkkaConfig.DispatcherThroughput, ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(prerequisites, name, settings.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
/** /**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. * 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) = def newBalancingDispatcher(name: String, throughput: Int) =
ThreadPoolConfigDispatcherBuilder(config 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. * 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) = def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config 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. * 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) = def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config 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 * Utility function that tries to load the specified dispatcher config from the akka.conf
* or else use the supplied default dispatcher * or else use the supplied default dispatcher
*/ */
def fromConfig(key: String, default: MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher = 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 * 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] = { def from(cfg: Configuration): Option[MessageDispatcher] = {
cfg.getString("type") flatMap { cfg.getString("type") flatMap {
case "Dispatcher" Some(new DispatcherConfigurator(app)) case "Dispatcher" Some(new DispatcherConfigurator())
case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator(app)) case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator())
case "GlobalDispatcher" None //TODO FIXME remove this
case fqn case fqn
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match { ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
case Right(clazz) case Right(clazz)
@ -197,31 +211,35 @@ class Dispatchers(val app: ActorSystem) {
throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn, exception) throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn, exception)
} }
} map { } map {
_ configure cfg _.configure(cfg, settings, prerequisites)
} }
} }
} }
class DispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) { class DispatcherConfigurator() extends MessageDispatcherConfigurator() {
def configure(config: Configuration): MessageDispatcher = { def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new Dispatcher(app, configureThreadPool(config,
config.getString("name", newUuid.toString), settings,
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput), threadPoolConfig new Dispatcher(prerequisites,
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), config.getString("name", newUuid.toString),
mailboxType(config), config.getInt("throughput", settings.DispatcherThroughput),
threadPoolConfig, config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build mailboxType(config, settings),
threadPoolConfig,
settings.DispatcherDefaultShutdown.toMillis)).build
} }
} }
class BalancingDispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) { class BalancingDispatcherConfigurator() extends MessageDispatcherConfigurator() {
def configure(config: Configuration): MessageDispatcher = { def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(app, configureThreadPool(config,
config.getString("name", newUuid.toString), settings,
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput), threadPoolConfig new BalancingDispatcher(prerequisites,
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), config.getString("name", newUuid.toString),
mailboxType(config), config.getInt("throughput", settings.DispatcherThroughput),
threadPoolConfig, config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build mailboxType(config, settings),
threadPoolConfig,
settings.DispatcherDefaultShutdown.toMillis)).build
} }
} }

View file

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

View file

@ -187,7 +187,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
} }
} catch { } catch {
case e 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 throw e
} }
} }

View file

@ -7,14 +7,27 @@ package akka.dispatch
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.actor.ActorSystem 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. * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long) class PinnedDispatcher(
extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) { _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 @volatile
protected[akka] var owner: ActorCell = _actor 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.event.Logging.{ Warning, Error }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import java.util.concurrent._ import java.util.concurrent._
import akka.event.EventStream
object ThreadPoolConfig { object ThreadPoolConfig {
type Bounds = Int 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 * A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
*/ */
case class ThreadPoolConfig(app: ActorSystem, case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize, corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
threadTimeout: Duration = ThreadPoolConfig.defaultTimeout, threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,

View file

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

View file

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

View file

@ -128,7 +128,7 @@ case class CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorExcept
override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter) override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter)
} }
abstract class RemoteSupport(val app: ActorSystem) { abstract class RemoteSupport(val system: ActorSystem) {
/** /**
* Shuts down the remoting * Shuts down the remoting
*/ */
@ -162,7 +162,7 @@ abstract class RemoteSupport(val app: ActorSystem) {
recipient: ActorRef, recipient: ActorRef,
loader: Option[ClassLoader]): Unit 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 override def toString = name
} }

View file

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

View file

@ -6,7 +6,7 @@ package akka.serialization
import akka.AkkaException import akka.AkkaException
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.actor.ActorSystem import akka.actor.{ ActorSystem, ActorSystemImpl }
import scala.util.DynamicVariable import scala.util.DynamicVariable
import akka.remote.RemoteSupport import akka.remote.RemoteSupport
@ -16,7 +16,7 @@ case class NoSerializerFoundException(m: String) extends AkkaException(m)
* Serialization module. Contains methods for serialization and deserialization as well as * Serialization module. Contains methods for serialization and deserialization as well as
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file. * locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
*/ */
class Serialization(val app: ActorSystem) { class Serialization(val system: ActorSystemImpl) {
//TODO document me //TODO document me
def serialize(o: AnyRef): Either[Exception, Array[Byte]] = def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
@ -28,7 +28,7 @@ class Serialization(val app: ActorSystem) {
clazz: Class[_], clazz: Class[_],
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] = classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
try { try {
Serialization.app.withValue(app) { Serialization.system.withValue(system) {
Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader))
} }
} catch { case e: Exception Left(e) } } catch { case e: Exception Left(e) }
@ -70,7 +70,7 @@ class Serialization(val app: ActorSystem) {
* But "default" can be overridden in config * But "default" can be overridden in config
*/ */
val serializers: Map[String, Serializer] = val serializers: Map[String, Serializer] =
app.config.getSection("akka.actor.serializers") system.settings.config.getSection("akka.actor.serializers")
.map(_.map) .map(_.map)
.getOrElse(Map()) .getOrElse(Map())
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) { .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 * 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]()) { _.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, (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? 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 { object Serialization {
// TODO ensure that these are always set (i.e. withValue()) when doing deserialization // TODO ensure that these are always set (i.e. withValue()) when doing deserialization
val app = new DynamicVariable[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 * 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) private val hasBooted = new Switch(false)
@volatile @volatile

View file

@ -278,7 +278,7 @@ abstract class Duration extends Serializable {
def /(other: Duration): Double def /(other: Duration): Double
def unary_- : Duration def unary_- : Duration
def finite_? : Boolean def finite_? : Boolean
def dilated(implicit app: 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 min(other: Duration): Duration = if (this < other) this else other
def max(other: Duration): Duration = if (this > other) this else other def max(other: Duration): Duration = if (this > other) this else other
def sleep(): Unit = Thread.sleep(toMillis) def sleep(): Unit = Thread.sleep(toMillis)

View file

@ -18,20 +18,20 @@ object JMX {
def nameFor(hostname: String, service: String, bean: String): ObjectName = def nameFor(hostname: String, service: String, bean: String): ObjectName =
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_"))) new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
def register(name: ObjectName, mbean: AnyRef)(implicit app: ActorSystem): Option[ObjectInstance] = try { def register(name: ObjectName, mbean: AnyRef)(implicit system: ActorSystem): Option[ObjectInstance] = try {
Some(mbeanServer.registerMBean(mbean, name)) Some(mbeanServer.registerMBean(mbean, name))
} catch { } catch {
case e: InstanceAlreadyExistsException case e: InstanceAlreadyExistsException
Some(mbeanServer.getObjectInstance(name)) Some(mbeanServer.getObjectInstance(name))
case e: Exception 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 None
} }
def unregister(mbean: ObjectName)(implicit app: ActorSystem) = try { def unregister(mbean: ObjectName)(implicit system: ActorSystem) = try {
mbeanServer.unregisterMBean(mbean) mbeanServer.unregisterMBean(mbean)
} catch { } catch {
case e: InstanceNotFoundException {} 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. * Starts this CamelService.
*/ */
def start: 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.initialized) CamelContextManager.init
if (!CamelContextManager.started) CamelContextManager.start 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 * 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 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 * 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 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 * @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. * 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 * 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 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 { "An non auto-acknowledging consumer" when {
"started" must { "started" must {
"must support acknowledgements on app level" in { "must support acknowledgements on system level" in {
var consumer: ActorRef = null var consumer: ActorRef = null
service.awaitEndpointActivation(1) { service.awaitEndpointActivation(1) {
consumer = actorOf(new TestAckConsumer("direct:app-ack-test")) consumer = actorOf(new TestAckConsumer("direct:system-ack-test"))
} must be(true) } 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 producer = endpoint.createProducer.asInstanceOf[AsyncProcessor]
val exchange = endpoint.createExchange 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 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 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 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 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. 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 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 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. new one for its use.

View file

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

View file

@ -11,10 +11,10 @@
// import java.util.concurrent.CountDownLatch // import java.util.concurrent.CountDownLatch
// //#imports // //#imports
// //#app // //#system
// object Pi extends App { // object Pi extends App {
// val app = ActorSystem() // val system = ActorSystem()
// calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) // calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
@ -66,10 +66,10 @@
// //#create-workers // //#create-workers
// // create the 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 // // 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 // //#create-workers
// //#master-receive // //#master-receive
@ -119,7 +119,7 @@
// val latch = new CountDownLatch(1) // val latch = new CountDownLatch(1)
// // create the master // // 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 // // start the calculation
// master ! Calculate // master ! Calculate
@ -128,5 +128,5 @@
// latch.await() // latch.await()
// } // }
// } // }
// //#app // //#system

View file

@ -14,7 +14,7 @@ import akka.config.Configuration
//#my-actor //#my-actor
class MyActor extends Actor { class MyActor extends Actor {
val log = Logging(app, this) val log = Logging(system, this)
def receive = { def receive = {
case "test" log.info("received test") case "test" log.info("received test")
case _ log.info("received unknown message") case _ log.info("received unknown message")
@ -36,8 +36,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
case e: Logging.Info true case e: Logging.Info true
case _ false case _ false
} }
app.eventStream.publish(TestEvent.Mute(filter)) system.eventStream.publish(TestEvent.Mute(filter))
app.eventStream.subscribe(testActor, classOf[Logging.Info]) system.eventStream.subscribe(testActor, classOf[Logging.Info])
myActor ! "test" myActor ! "test"
expectMsgPF(1 second) { case Logging.Info(_, "received test") true } expectMsgPF(1 second) { case Logging.Info(_, "received test") true }
@ -45,8 +45,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
myActor ! "unknown" myActor ! "unknown"
expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true } expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true }
app.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
app.eventStream.publish(TestEvent.UnMute(filter)) system.eventStream.publish(TestEvent.UnMute(filter))
myActor.stop() 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 { 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 hostname = system.settings.config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0")
val port = app.config.getInt("akka.actor.mailbox.beanstalk.port", 11300) val port = system.settings.config.getInt("akka.actor.mailbox.beanstalk.port", 11300)
def defaultTimeUnit = app.AkkaConfig.DefaultTimeUnit def defaultTimeUnit = system.settings.DefaultTimeUnit
val reconnectWindow = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), defaultTimeUnit).toSeconds.toInt val reconnectWindow = Duration(system.settings.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 messageSubmitDelay = Duration(system.settings.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 messageSubmitTimeout = Duration(system.settings.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 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) } 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 { 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 { private val queue = try {
try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case e {} } 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.setup // replays journal
queue.discardExpired queue.discardExpired
queue queue

View file

@ -142,11 +142,11 @@ object QDumper {
System.exit(0) System.exit(0)
} }
val app = ActorSystem() val system = ActorSystem()
for (filename filenames) { for (filename filenames) {
println("Queue: " + filename) 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) { class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileDurableMailboxType) {
def clean { def clean {
val queuePath = FileBasedMailbox.queuePath(app.config) val queuePath = FileBasedMailbox.queuePath(system.settings.config)
FileUtils.deleteDirectory(new java.io.File(queuePath)) 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 { abstract class DurableMailbox(owner: ActorCell) extends Mailbox(owner) with DefaultSystemMessageQueue {
import DurableExecutableMailboxConfig._ import DurableExecutableMailboxConfig._
def app = owner.app def system = owner.system
def ownerPath = owner.self.path def ownerPath = owner.self.path
val ownerPathString = ownerPath.path.mkString("/") val ownerPathString = ownerPath.path.mkString("/")
val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_") val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
@ -54,11 +54,11 @@ trait DurableMessageSerialization {
def serialize(durableMessage: Envelope): Array[Byte] = { def serialize(durableMessage: Envelope): Array[Byte] = {
def serializeActorRef(ref: ActorRef): ActorRefProtocol = { 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 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 val builder = RemoteMessageProtocol.newBuilder
.setMessage(message) .setMessage(message)
.setRecipient(serializeActorRef(owner.self)) .setRecipient(serializeActorRef(owner.self))
@ -71,11 +71,11 @@ trait DurableMessageSerialization {
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = { def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = {
val serRef = SerializedActorRef(refProtocol.getHost, refProtocol.getPort, refProtocol.getPath) 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 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) val sender = deserializeActorRef(durableMessage.getSender)
new Envelope(message, sender) 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 { abstract class DurableMailboxSpec(val backendName: String, val mailboxType: DurableMailboxType) extends AkkaSpec with BeforeAndAfterEach {
import DurableMailboxSpecActorFactory._ 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 = def createMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef =
actorOf(Props(new MailboxTestActor).withDispatcher(dispatcher)) actorOf(Props(new MailboxTestActor).withDispatcher(dispatcher))

View file

@ -17,9 +17,11 @@ import org.bson.DefaultBSONSerializer
import akka.actor.SerializedActorRef import akka.actor.SerializedActorRef
import akka.remote.RemoteProtocol.MessageProtocol import akka.remote.RemoteProtocol.MessageProtocol
import akka.remote.MessageSerializer 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) = { 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 += "_id" -> msg._id
b += "ownerPath" -> msg.ownerPath b += "ownerPath" -> msg.ownerPath
val sender = app.provider.serialize(msg.sender) val sender = systemImpl.provider.serialize(msg.sender)
b += "senderPath" -> sender.path b += "senderPath" -> sender.path
b += "senderHostname" -> sender.hostname b += "senderHostname" -> sender.hostname
b += "senderPort" -> sender.port 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 - 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 * 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) b += "message" -> new org.bson.types.Binary(0, msgData.toByteArray)
val doc = b.result val doc = b.result
app.log.debug("Serialized Document: {}", doc) system.log.debug("Serialized Document: {}", doc)
serializer.putObject(doc) serializer.putObject(doc)
} }
@ -68,15 +70,15 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
val deserializer = new DefaultBSONDeserializer val deserializer = new DefaultBSONDeserializer
// TODO - Skip the whole doc step for performance, fun, and profit! (Needs Salat / custom Deser) // TODO - Skip the whole doc step for performance, fun, and profit! (Needs Salat / custom Deser)
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] 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 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 ownerPath = doc.as[String]("ownerPath")
val senderPath = doc.as[String]("senderPath") val senderPath = doc.as[String]("senderPath")
val senderHostname = doc.as[String]("senderHostname") val senderHostname = doc.as[String]("senderHostname")
val senderPort = doc.as[Int]("senderPort") val senderPort = doc.as[Int]("senderPort")
val sender = app.provider.deserialize(SerializedActorRef(senderHostname, senderPort, senderPath)). val sender = systemImpl.provider.deserialize(SerializedActorRef(senderHostname, senderPort, senderPath)).
getOrElse(app.deadLetters) getOrElse(system.deadLetters)
MongoDurableMessage(ownerPath, msg, sender) 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) { class MongoBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) {
// this implicit object provides the context for reading/writing things as MongoDurableMessage // 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! implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
val URI_CONFIG_KEY = "akka.actor.mailbox.mongodb.uri" val URI_CONFIG_KEY = "akka.actor.mailbox.mongodb.uri"
val WRITE_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.write" val WRITE_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.write"
val READ_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.read" val READ_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.read"
val mongoURI = app.config.getString(URI_CONFIG_KEY) val mongoURI = system.settings.config.getString(URI_CONFIG_KEY)
val writeTimeout = app.config.getInt(WRITE_TIMEOUT_KEY, 3000) val writeTimeout = system.settings.config.getInt(WRITE_TIMEOUT_KEY, 3000)
val readTimeout = app.config.getInt(READ_TIMEOUT_KEY, 3000) val readTimeout = system.settings.config.getInt(READ_TIMEOUT_KEY, 3000)
val log = Logging(app, this) val log = Logging(system, this)
@volatile @volatile
private var mongo = connect() private var mongo = connect()

View file

@ -20,7 +20,7 @@ class RedisBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with
@volatile @volatile
private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling 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) { def enqueue(receiver: ActorRef, envelope: Envelope) {
log.debug("ENQUEUING message in redis-based mailbox [%s]".format(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() = { private[akka] def connect() = {
new RedisClientPool( new RedisClientPool(
app.config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"), system.settings.config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"),
app.config.getInt("akka.actor.mailbox.redis.port", 6379)) system.settings.config.getInt("akka.actor.mailbox.redis.port", 6379))
} }
private def withErrorHandling[T](body: T): T = { 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 { class ZooKeeperBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
val zkServerAddresses = app.config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181") val zkServerAddresses = system.settings.config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181")
def defaultTimeUnit = app.AkkaConfig.DefaultTimeUnit def defaultTimeUnit = system.settings.DefaultTimeUnit
val sessionTimeout = Duration(app.config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), defaultTimeUnit).toMillis.toInt val sessionTimeout = Duration(system.settings.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 connectionTimeout = Duration(system.settings.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 blockingQueue = system.settings.config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true)
val queueNode = "/queues" val queueNode = "/queues"
val queuePathTemplate = queueNode + "/%s" val queuePathTemplate = queueNode + "/%s"
val log = Logging(app, this) val log = Logging(system, this)
private val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout) private val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout)
private val queue = new ZooKeeperQueue[Array[Byte]](zkClient, queuePathTemplate.format(name), blockingQueue) 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. * This class can be added to web.xml mappings as a listener to start and postStop Akka.
* *
* <web-app> * <web-system>
* ... * ...
* <listener> * <listener>
* <listener-class>akka.servlet.Initializer</listener-class> * <listener-class>akka.servlet.Initializer</listener-class>
* </listener> * </listener>
* ... * ...
* </web-app> * </web-system>
*/ */
class Initializer extends ServletContextListener { class Initializer extends ServletContextListener {
lazy val loader = new AkkaLoader 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) { class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 1000) {
def this(app: ActorSystem) { def this(system: ActorSystem) {
this( this(
app.config.getInt("akka.remote.failure-detector.theshold", 8), system.settings.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.max-sample-size", 1000))
} }
private final val PhiFactor = 1.0 / math.log(10.0) private final val PhiFactor = 1.0 / math.log(10.0)

View file

@ -21,26 +21,26 @@ trait BootableRemoteActorService extends Bootable {
def settings: RemoteServerSettings def settings: RemoteServerSettings
protected lazy val remoteServerThread = new Thread(new Runnable() { 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") }, "Akka RemoteModule Service")
def startRemoteService() { remoteServerThread.start() } def startRemoteService() { remoteServerThread.start() }
abstract override def onLoad() { abstract override def onLoad() {
if (app.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) { if (system.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
app.eventHandler.info(this, "Initializing Remote Actors Service...") system.eventHandler.info(this, "Initializing Remote Actors Service...")
startRemoteService() startRemoteService()
app.eventHandler.info(this, "Remote Actors Service initialized") system.eventHandler.info(this, "Remote Actors Service initialized")
} }
super.onLoad() super.onLoad()
} }
abstract override def onUnload() { 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) 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() super.onUnload()
} }
} }

View file

@ -101,13 +101,13 @@ class Gossiper(remote: Remote) {
currentGossip: Gossip, currentGossip: Gossip,
nodeMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener]) nodeMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener])
private val app = remote.app private val system = remote.system
private val log = Logging(app, this) private val log = Logging(system, this)
private val failureDetector = remote.failureDetector 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 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 nodeFingerprint = address.##
private val random = SecureRandom.getInstance("SHA1PRNG") 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 // 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) system.scheduler schedule (() initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
app.scheduler schedule (() scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit) system.scheduler schedule (() scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
} }
/** /**
@ -153,7 +153,7 @@ class Gossiper(remote: Remote) {
node oldAvailableNodes node oldAvailableNodes
if connectionManager.connectionFor(node).isEmpty 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 connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes 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 = { 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 Left(error) throw error
case Right(bytes) bytes case Right(bytes) bytes
} }

View file

@ -11,15 +11,15 @@ import akka.actor.ActorSystem
object MessageSerializer { 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) val clazz = loadManifest(classLoader, messageProtocol)
app.serialization.deserialize(messageProtocol.getMessage.toByteArray, system.serialization.deserialize(messageProtocol.getMessage.toByteArray,
clazz, classLoader).fold(x throw x, identity) 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 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.setMessage(ByteString.copyFrom(bytes))
builder.setMessageManifest(ByteString.copyFromUtf8(message.getClass.getName)) builder.setMessageManifest(ByteString.copyFromUtf8(message.getClass.getName))
builder.build builder.build

View file

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

View file

@ -15,33 +15,30 @@ import akka.actor.DeploymentConfig._
import akka.serialization.Compression.LZF import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress import java.net.InetSocketAddress
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression } import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression }
import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher } 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. * 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> * @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 system._
import app.config val AC = settings
import app.AkkaConfig._ import AC._
val nodename = app.nodename // TODO move to settings?
// TODO move to AkkaConfig?
val shouldCompressData = config.getBool("akka.remote.use-compression", false) 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 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) // val gossiper = new Gossiper(this)
@ -50,17 +47,18 @@ class Remote(val app: ActorSystem) {
// FIXME configure computeGridDispatcher to what? // FIXME configure computeGridDispatcher to what?
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build 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? OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want?
private[remote] lazy val remoteDaemon = private[remote] lazy val remoteDaemon =
app.provider.actorOf( system.provider.actorOf(system,
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)), Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
remoteDaemonSupervisor, remoteDaemonSupervisor,
remoteDaemonServiceName, remoteDaemonServiceName,
systemService = true) systemService = true)
private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor { private[remote] lazy val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor {
def receive = { def receive = {
case RemoteClientError(cause, remote, address) remote.shutdownClientConnection(address) case RemoteClientError(cause, remote, address) remote.shutdownClientConnection(address)
case RemoteClientDisconnected(remote, address) remote.shutdownClientConnection(address) case RemoteClientDisconnected(remote, address) remote.shutdownClientConnection(address)
@ -68,22 +66,22 @@ class Remote(val app: ActorSystem) {
} }
}), "akka.remote.RemoteClientLifeCycleListener") }), "akka.remote.RemoteClientLifeCycleListener")
lazy val eventStream = new NetworkEventStream(app) lazy val eventStream = new NetworkEventStream(system)
lazy val server: RemoteSupport = { 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? remote.start() //TODO FIXME Any application loader here?
app.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent]) system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
app.eventStream.subscribe(remoteClientLifeCycleHandler, 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) //provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
remote remote
} }
def start(): Unit = { 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 val daemonAddress = remoteDaemon.address //Force init of daemon
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress) 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 { class RemoteSystemDaemon(remote: Remote) extends Actor {
import remote._ import remote._
import remote.{ system systemImpl }
override def preRestart(reason: Throwable, msg: Option[Any]) { override def preRestart(reason: Throwable, msg: Option[Any]) {
log.debug("RemoteSystemDaemon failed due to [{}] - restarting...", reason) 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 if (shouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
val actorFactory = val actorFactory =
app.serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match { system.serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match {
case Left(error) throw error case Left(error) throw error
case Right(instance) instance.asInstanceOf[() Actor] case Right(instance) instance.asInstanceOf[() Actor]
} }
val actorPath = ActorPath(remote.app, message.getActorPath) val actorPath = ActorPath(systemImpl, message.getActorPath)
val parent = actorPath.parent.ref val parent = system.actorFor(actorPath.parent)
if (parent.isDefined) { 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 { } else {
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message) 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) 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 { } catch {
case error: Throwable //FIXME doesn't seem sensible case error: Throwable //FIXME doesn't seem sensible
sender ! Failure(error) 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 // FIXME: handle real remote supervision
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) { def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(systemImpl,
Props( Props(
context { context {
case f: Function0[_] try { f() } finally { context.self.stop() } 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 // FIXME: handle real remote supervision
def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) { def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(systemImpl,
Props( Props(
context { context {
case f: Function0[_] try { sender ! f() } finally { context.self.stop() } 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 // FIXME: handle real remote supervision
def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) { def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(systemImpl,
Props( Props(
context { context {
case (fun: Function[_, _], param: Any) try { fun.asInstanceOf[Any Unit].apply(param) } finally { context.self.stop() } 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 // FIXME: handle real remote supervision
def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) { def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(systemImpl,
Props( Props(
context { context {
case (fun: Function[_, _], param: Any) try { sender ! fun.asInstanceOf[Any Any](param) } finally { context.self.stop() } 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) { def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
@ -224,7 +230,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
} }
private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = { 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 Left(error) throw error
case Right(instance) instance.asInstanceOf[T] 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) { class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
val provider = remote.system.asInstanceOf[ActorSystemImpl].provider
lazy val sender: ActorRef = lazy val sender: ActorRef =
if (input.hasSender) if (input.hasSender)
remote.app.provider.deserialize( provider.deserialize(
SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES")) SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES"))
else 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] = lazy val payload: Either[Throwable, AnyRef] =
if (input.hasException) Left(parseException()) 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 = { protected def parseException(): Throwable = {
val exception = input.getException val exception = input.getException
@ -256,7 +265,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
.newInstance(exception.getMessage).asInstanceOf[Throwable] .newInstance(exception.getMessage).asInstanceOf[Throwable]
} catch { } catch {
case problem: Exception 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) CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
} }
} }
@ -266,7 +275,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
trait RemoteMarshallingOps { trait RemoteMarshallingOps {
def app: ActorSystem def system: ActorSystem
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
val arp = AkkaRemoteProtocol.newBuilder val arp = AkkaRemoteProtocol.newBuilder
@ -284,7 +293,7 @@ trait RemoteMarshallingOps {
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
*/ */
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { 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 ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build
} }
@ -297,7 +306,7 @@ trait RemoteMarshallingOps {
message match { message match {
case Right(message) case Right(message)
messageBuilder.setMessage(MessageSerializer.serialize(app, message.asInstanceOf[AnyRef])) messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
case Left(exception) case Left(exception)
messageBuilder.setException(ExceptionProtocol.newBuilder messageBuilder.setException(ExceptionProtocol.newBuilder
.setClassname(exception.getClass.getName) .setClassname(exception.getClass.getName)

View file

@ -17,47 +17,63 @@ import akka.serialization.{ Serialization, Serializer, Compression }
import akka.serialization.Compression.LZF import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import com.google.protobuf.ByteString import com.google.protobuf.ByteString
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import akka.event.EventStream
/** /**
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. * 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> * @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 java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise import akka.dispatch.Promise
val local = new LocalActorRefProvider(app) val local = new LocalActorRefProvider(settings, rootPath, eventStream, dispatcher, scheduler)
val remote = new Remote(app) 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 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 theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime
private[akka] def terminationFuture = local.terminationFuture private[akka] def terminationFuture = local.terminationFuture
private[akka] def deployer: Deployer = local.deployer private[akka] def deployer: Deployer = local.deployer
def defaultDispatcher = app.dispatcher def defaultDispatcher = dispatcher
def defaultTimeout = app.AkkaConfig.ActorTimeout 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 = private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
actorOf(props, supervisor, supervisor.path / name, systemService) if (systemService) local.actorOf(system, props, supervisor, path, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
if (systemService) local.actorOf(props, supervisor, path, systemService)
else { else {
val name = path.name val name = path.name
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? 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) // 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) { if (isReplicaNode) {
// we are on one of the replica node for this remote actor // 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 { } else {
// we are on the single "reference" node uses the remote actors on the replica nodes // 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 connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a)
val remoteAddress = RemoteAddress(a.hostname, a.port) 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 { } catch {
case e: Exception case e: Exception
@ -137,7 +153,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
throw e 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 newFuture completeWithResult actor
actors.replace(path.toString, newFuture, actor) actors.replace(path.toString, newFuture, actor)
@ -151,9 +167,9 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
* Copied from LocalActorRefProvider... * Copied from LocalActorRefProvider...
*/ */
// FIXME: implement supervision // 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") 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 { 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]) case future: Future[_] Some(future.get.asInstanceOf[ActorRef])
} }
// TODO remove me
val optimizeLocal = new AtomicBoolean(true) val optimizeLocal = new AtomicBoolean(true)
def optimizeLocalScoped_?() = optimizeLocal.get def optimizeLocalScoped_?() = optimizeLocal.get
@ -177,22 +194,22 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = { private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
val remoteAddress = RemoteAddress(actor.hostname, actor.port) val remoteAddress = RemoteAddress(actor.hostname, actor.port)
if (optimizeLocalScoped_? && remoteAddress == app.address) { if (optimizeLocalScoped_? && remoteAddress == rootPath.remoteAddress) {
local.actorFor(ActorPath.split(actor.path)) local.actorFor(ActorPath.split(actor.path))
} else { } else {
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress) log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", rootPath.remoteAddress, actor.path, remoteAddress)
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here 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. * Using (checking out) actor on a specific node.
*/ */
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) { def useActorOnNode(system: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) {
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress) log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress)
val actorFactoryBytes = val actorFactoryBytes =
app.serialization.serialize(actorFactory) match { system.serialization.serialize(actorFactory) match {
case Left(error) throw error case Left(error) throw error
case Right(bytes) if (remote.shouldCompressData) LZF.compress(bytes) else bytes 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 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 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
private[akka] case class RemoteActorRef private[akka] ( private[akka] case class RemoteActorRef private[akka] (
provider: ActorRefProvider,
remote: RemoteSupport, remote: RemoteSupport,
remoteAddress: RemoteAddress, remoteAddress: RemoteAddress,
path: ActorPath, 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 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 = () def suspend(): Unit = ()
@ -289,7 +305,7 @@ private[akka] case class RemoteActorRef private[akka] (
} }
@throws(classOf[java.io.ObjectStreamException]) @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 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class RemoteConnectionManager( class RemoteConnectionManager(
app: ActorSystem, system: ActorSystem,
remote: Remote, remote: Remote,
initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef]) initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef])
extends ConnectionManager { extends ConnectionManager {
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. // 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]) case class State(version: Long, connections: Map[RemoteAddress, ActorRef])
@ -149,5 +149,5 @@ class RemoteConnectionManager(
} }
private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) = 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 remoteSupport: NettyRemoteSupport,
val remoteAddress: RemoteAddress) { val remoteAddress: RemoteAddress) {
val log = Logging(remoteSupport.app, this) val log = Logging(remoteSupport.system, this)
val name = simpleName(this) + "@" + remoteAddress val name = simpleName(this) + "@" + remoteAddress
@ -148,7 +148,8 @@ class ActiveRemoteClient private[akka] (
def sendSecureCookie(connection: ChannelFuture) { def sendSecureCookie(connection: ChannelFuture) {
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get) 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)) connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
} }
@ -349,10 +350,10 @@ class ActiveRemoteClientHandler(
/** /**
* Provides the implementation of the Netty remote support * 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 serverSettings = new RemoteServerSettings(system.settings.config, system.settings.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit) val clientSettings = new RemoteClientSettings(system.settings.config, system.settings.DefaultTimeUnit)
private val remoteClients = new HashMap[RemoteAddress, RemoteClient] private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val clientsLock = new ReentrantReadWriteLock private val clientsLock = new ReentrantReadWriteLock
@ -447,7 +448,7 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
def name = currentServer.get match { def name = currentServer.get match {
case Some(server) server.name 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) 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]) { 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.serverSettings._
import remoteSupport.app.address
val address = remoteSupport.system.rootPath.remoteAddress
val name = "NettyRemoteServer@" + address val name = "NettyRemoteServer@" + address
@ -583,7 +585,7 @@ class RemoteServerHandler(
val applicationLoader: Option[ClassLoader], val applicationLoader: Option[ClassLoader],
val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler { val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler {
val log = Logging(remoteSupport.app, this) val log = Logging(remoteSupport.system, this)
import remoteSupport.serverSettings._ import remoteSupport.serverSettings._

View file

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

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