Adding DispatcherPrerequisites to hold the common dependencies that a dispatcher needs to be created
This commit is contained in:
commit
80d766b07b
139 changed files with 1126 additions and 948 deletions
|
|
@ -9,15 +9,15 @@ import static org.junit.Assert.*;
|
|||
|
||||
public class JavaAPI {
|
||||
|
||||
private ActorSystem app = new ActorSystem();
|
||||
private ActorSystem system = ActorSystem.create();
|
||||
|
||||
@Test void mustBeAbleToCreateActorRefFromClass() {
|
||||
ActorRef ref = app.actorOf(JavaAPITestActor.class);
|
||||
ActorRef ref = system.actorOf(JavaAPITestActor.class);
|
||||
assertNotNull(ref);
|
||||
}
|
||||
|
||||
@Test void mustBeAbleToCreateActorRefFromFactory() {
|
||||
ActorRef ref = app.actorOf(new Props().withCreator(new Creator<Actor>() {
|
||||
ActorRef ref = system.actorOf(new Props().withCreator(new Creator<Actor>() {
|
||||
public Actor create() {
|
||||
return new JavaAPITestActor();
|
||||
}
|
||||
|
|
@ -26,7 +26,7 @@ public class JavaAPI {
|
|||
}
|
||||
|
||||
@Test void mustAcceptSingleArgTell() {
|
||||
ActorRef ref = app.actorOf(JavaAPITestActor.class);
|
||||
ActorRef ref = system.actorOf(JavaAPITestActor.class);
|
||||
ref.tell("hallo");
|
||||
ref.tell("hallo", ref);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,9 +19,9 @@ import scala.Right;
|
|||
|
||||
public class JavaFutureTests {
|
||||
|
||||
private final ActorSystem app = new ActorSystem();
|
||||
private final Timeout t = app.AkkaConfig().ActorTimeout();
|
||||
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);
|
||||
private final ActorSystem system = ActorSystem.create();
|
||||
private final Timeout t = system.settings().ActorTimeout();
|
||||
private final FutureFactory ff = new FutureFactory(system.dispatcher(), t);
|
||||
|
||||
@Test public void mustBeAbleToMapAFuture() {
|
||||
Future<String> f1 = ff.future(new Callable<String>() {
|
||||
|
|
@ -41,7 +41,7 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToExecuteAnOnResultCallback() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Future<String> f = cf;
|
||||
f.onResult(new Procedure<String>() {
|
||||
public void apply(String result) {
|
||||
|
|
@ -57,7 +57,7 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToExecuteAnOnExceptionCallback() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Future<String> f = cf;
|
||||
f.onException(new Procedure<Throwable>() {
|
||||
public void apply(Throwable t) {
|
||||
|
|
@ -74,7 +74,7 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToExecuteAnOnTimeoutCallback() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Future<String> f = cf;
|
||||
f.onTimeout(new Procedure<Future<String>>() {
|
||||
public void apply(Future<String> future) {
|
||||
|
|
@ -88,7 +88,7 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToExecuteAnOnCompleteCallback() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Future<String> f = cf;
|
||||
f.onComplete(new Procedure<Future<String>>() {
|
||||
public void apply(akka.dispatch.Future<String> future) {
|
||||
|
|
@ -103,7 +103,7 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToForeachAFuture() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Future<String> f = cf;
|
||||
f.foreach(new Procedure<String>() {
|
||||
public void apply(String future) {
|
||||
|
|
@ -118,13 +118,13 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToFlatMapAFuture() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
cf.completeWithResult("1000");
|
||||
Future<String> f = cf;
|
||||
Future<Integer> r = f.flatMap(new Function<String, Future<Integer>>() {
|
||||
public Future<Integer> apply(String r) {
|
||||
latch.countDown();
|
||||
Promise<Integer> cf = new akka.dispatch.DefaultPromise<Integer>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<Integer> cf = new akka.dispatch.DefaultPromise<Integer>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
cf.completeWithResult(Integer.parseInt(r));
|
||||
return cf;
|
||||
}
|
||||
|
|
@ -137,7 +137,7 @@ public class JavaFutureTests {
|
|||
|
||||
@Test public void mustBeAbleToFilterAFuture() throws Throwable {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Promise<String> cf = new akka.dispatch.DefaultPromise<String>(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher());
|
||||
Future<String> f = cf;
|
||||
Future<String> r = f.filter(new Function<String, Boolean>() {
|
||||
public Boolean apply(String r) {
|
||||
|
|
|
|||
|
|
@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec {
|
|||
out.flush
|
||||
out.close
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
val readA = in.readObject
|
||||
|
||||
|
|
@ -257,7 +257,7 @@ class ActorRefSpec extends AkkaSpec {
|
|||
}
|
||||
}
|
||||
|
||||
"throw an exception on deserialize if no app in scope" in {
|
||||
"throw an exception on deserialize if no system in scope" in {
|
||||
val a = actorOf[InnerActor]
|
||||
|
||||
import java.io._
|
||||
|
|
@ -275,7 +275,7 @@ class ActorRefSpec extends AkkaSpec {
|
|||
(intercept[java.lang.IllegalStateException] {
|
||||
in.readObject
|
||||
}).getMessage must be === "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
|
||||
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }"
|
||||
" Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }"
|
||||
}
|
||||
|
||||
"must throw exception on deserialize if not present in actor hierarchy (and remoting is not enabled)" in {
|
||||
|
|
@ -284,14 +284,15 @@ class ActorRefSpec extends AkkaSpec {
|
|||
val baos = new ByteArrayOutputStream(8192 * 32)
|
||||
val out = new ObjectOutputStream(baos)
|
||||
|
||||
val serialized = SerializedActorRef(app.address.hostname, app.address.port, "/this/path/does/not/exist")
|
||||
val addr = system.rootPath.remoteAddress
|
||||
val serialized = SerializedActorRef(addr.hostname, addr.port, "/this/path/does/not/exist")
|
||||
|
||||
out.writeObject(serialized)
|
||||
|
||||
out.flush
|
||||
out.close
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
(intercept[java.lang.IllegalStateException] {
|
||||
in.readObject
|
||||
|
|
|
|||
|
|
@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
|
|||
}
|
||||
}, timeout = t))
|
||||
|
||||
val defaultTimeout = app.AkkaConfig.ActorTimeout.duration
|
||||
val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
|
||||
val defaultTimeout = system.settings.ActorTimeout.duration
|
||||
val testTimeout = if (system.settings.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
|
||||
|
||||
"An Actor-based Future" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ class ClusterSpec extends AkkaSpec {
|
|||
"be able to parse 'akka.actor.cluster._' config elements" in {
|
||||
|
||||
// TODO: make it use its own special config?
|
||||
val config = app.config
|
||||
val config = system.settings.config
|
||||
import config._
|
||||
|
||||
//akka.cluster
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ class DeployerSpec extends AkkaSpec {
|
|||
|
||||
"A Deployer" must {
|
||||
"be able to parse 'akka.actor.deployment._' config elements" in {
|
||||
val deployment = app.provider.deployer.lookupInConfig("/app/service-ping")
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookupInConfig("/app/service-ping")
|
||||
deployment must be('defined)
|
||||
|
||||
deployment must equal(Some(
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import akka.config.Configuration
|
|||
|
||||
object FSMActorSpec {
|
||||
|
||||
class Latches(implicit app: ActorSystem) {
|
||||
class Latches(implicit system: ActorSystem) {
|
||||
val unlockedLatch = TestLatch()
|
||||
val lockedLatch = TestLatch()
|
||||
val unhandledLatch = TestLatch()
|
||||
|
|
@ -167,12 +167,12 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
|
|||
}
|
||||
})
|
||||
filterException[Logging.EventHandlerException] {
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Error])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Error])
|
||||
fsm ! "go"
|
||||
expectMsgPF(1 second, hint = "Next state 2 does not exist") {
|
||||
case Logging.Error(_, `fsm`, "Next state 2 does not exist") ⇒ true
|
||||
}
|
||||
app.eventStream.unsubscribe(testActor)
|
||||
system.eventStream.unsubscribe(testActor)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -213,20 +213,20 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
|
|||
case StopEvent(r, _, _) ⇒ testActor ! r
|
||||
}
|
||||
})
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
fsm ! "go"
|
||||
expectMsgPF(1 second, hint = "processing Event(go,null)") {
|
||||
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[" + app.address + "/sys/testActor") ⇒ true
|
||||
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[") ⇒ true
|
||||
}
|
||||
expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown"))
|
||||
expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2"))
|
||||
fsm ! "stop"
|
||||
expectMsgPF(1 second, hint = "processing Event(stop,null)") {
|
||||
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[" + app.address + "/sys/testActor") ⇒ true
|
||||
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[") ⇒ true
|
||||
}
|
||||
expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal)
|
||||
expectNoMsg(1 second)
|
||||
app.eventStream.unsubscribe(testActor)
|
||||
system.eventStream.unsubscribe(testActor)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
|
||||
"A Finite State Machine" must {
|
||||
|
||||
"receive StateTimeout" in {
|
||||
"receive StateTimeout" taggedAs TimingTest in {
|
||||
within(1 second) {
|
||||
within(500 millis, 1 second) {
|
||||
fsm ! TestStateTimeout
|
||||
|
|
@ -34,7 +34,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
}
|
||||
|
||||
"cancel a StateTimeout" in {
|
||||
"cancel a StateTimeout" taggedAs TimingTest in {
|
||||
within(1 second) {
|
||||
fsm ! TestStateTimeout
|
||||
fsm ! Cancel
|
||||
|
|
@ -44,7 +44,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
}
|
||||
|
||||
"allow StateTimeout override" in {
|
||||
"allow StateTimeout override" taggedAs TimingTest in {
|
||||
within(500 millis) {
|
||||
fsm ! TestStateTimeoutOverride
|
||||
expectNoMsg
|
||||
|
|
@ -56,8 +56,8 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
}
|
||||
|
||||
"receive single-shot timer" in {
|
||||
within(1.5 seconds) {
|
||||
"receive single-shot timer" taggedAs TimingTest in {
|
||||
within(2 seconds) {
|
||||
within(500 millis, 1 second) {
|
||||
fsm ! TestSingleTimer
|
||||
expectMsg(Tick)
|
||||
|
|
@ -67,7 +67,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
}
|
||||
|
||||
"correctly cancel a named timer" in {
|
||||
"correctly cancel a named timer" taggedAs TimingTest in {
|
||||
fsm ! TestCancelTimer
|
||||
within(500 millis) {
|
||||
fsm ! Tick
|
||||
|
|
@ -80,7 +80,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
expectMsg(1 second, Transition(fsm, TestCancelTimer, Initial))
|
||||
}
|
||||
|
||||
"not get confused between named and state timers" in {
|
||||
"not get confused between named and state timers" taggedAs TimingTest in {
|
||||
fsm ! TestCancelStateTimerInNamedTimerMessage
|
||||
fsm ! Tick
|
||||
expectMsg(500 millis, Tick)
|
||||
|
|
@ -94,7 +94,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
}
|
||||
|
||||
"receive and cancel a repeated timer" in {
|
||||
"receive and cancel a repeated timer" taggedAs TimingTest in {
|
||||
fsm ! TestRepeatedTimer
|
||||
val seq = receiveWhile(2 seconds) {
|
||||
case Tick ⇒ Tick
|
||||
|
|
@ -105,7 +105,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
}
|
||||
|
||||
"notify unhandled messages" in {
|
||||
"notify unhandled messages" taggedAs TimingTest in {
|
||||
filterEvents(EventFilter.warning("unhandled event Tick in state TestUnhandled", source = fsm, occurrences = 1),
|
||||
EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm, occurrences = 1)) {
|
||||
fsm ! TestUnhandled
|
||||
|
|
|
|||
|
|
@ -12,12 +12,12 @@ import akka.util.Duration
|
|||
object ForwardActorSpec {
|
||||
val ExpectedMessage = "FOO"
|
||||
|
||||
def createForwardingChain(app: ActorSystem): ActorRef = {
|
||||
val replier = app.actorOf(new Actor {
|
||||
def createForwardingChain(system: ActorSystem): ActorRef = {
|
||||
val replier = system.actorOf(new Actor {
|
||||
def receive = { case x ⇒ sender ! x }
|
||||
})
|
||||
|
||||
def mkforwarder(forwardTo: ActorRef) = app.actorOf(
|
||||
def mkforwarder(forwardTo: ActorRef) = system.actorOf(
|
||||
new Actor {
|
||||
def receive = { case x ⇒ forwardTo forward x }
|
||||
})
|
||||
|
|
@ -37,14 +37,14 @@ class ForwardActorSpec extends AkkaSpec {
|
|||
|
||||
val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } })
|
||||
|
||||
val chain = createForwardingChain(app)
|
||||
val chain = createForwardingChain(system)
|
||||
|
||||
chain.tell(ExpectedMessage, replyTo)
|
||||
latch.await(Duration(5, "s")) must be === true
|
||||
}
|
||||
|
||||
"forward actor reference when invoking forward on bang bang" in {
|
||||
val chain = createForwardingChain(app)
|
||||
val chain = createForwardingChain(system)
|
||||
chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,14 +13,15 @@ class LocalActorRefProviderSpec extends AkkaSpec {
|
|||
"An LocalActorRefProvider" must {
|
||||
|
||||
"only create one instance of an actor with a specific address in a concurrent environment" in {
|
||||
val provider = app.provider
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
val provider = impl.provider
|
||||
|
||||
provider.isInstanceOf[LocalActorRefProvider] must be(true)
|
||||
|
||||
(0 until 100) foreach { i ⇒ // 100 concurrent runs
|
||||
val address = "new-actor" + i
|
||||
implicit val timeout = Timeout(5 seconds)
|
||||
((1 to 4) map { _ ⇒ Future { provider.actorOf(Props(c ⇒ { case _ ⇒ }), app.guardian, address) } }).map(_.get).distinct.size must be(1)
|
||||
((1 to 4) map { _ ⇒ Future { provider.actorOf(impl, Props(c ⇒ { case _ ⇒ }), impl.guardian, address) } }).map(_.get).distinct.size must be(1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -53,7 +53,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
|
||||
"decorate a Receive" in {
|
||||
new TestKit(appLogging) {
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
val r: Actor.Receive = {
|
||||
case null ⇒
|
||||
}
|
||||
|
|
@ -66,8 +66,8 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
"be added on Actor if requested" in {
|
||||
new TestKit(appLogging) with ImplicitSender {
|
||||
ignoreMute(this)
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Error])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Error])
|
||||
val actor = TestActorRef(new Actor {
|
||||
def receive = loggable(this) {
|
||||
case _ ⇒ sender ! "x"
|
||||
|
|
@ -95,7 +95,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
|
||||
"not duplicate logging" in {
|
||||
new TestKit(appLogging) with ImplicitSender {
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
val actor = TestActorRef(new Actor {
|
||||
def receive = loggable(this)(loggable(this) {
|
||||
case _ ⇒ sender ! "x"
|
||||
|
|
@ -115,7 +115,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
|
||||
"log AutoReceiveMessages if requested" in {
|
||||
new TestKit(appAuto) {
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
val actor = TestActorRef(new Actor {
|
||||
def receive = {
|
||||
case _ ⇒
|
||||
|
|
@ -137,10 +137,10 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
val s = ref.toString
|
||||
s.contains("MainBusReaper") || s.contains("Supervisor")
|
||||
}
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Error])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Error])
|
||||
within(3 seconds) {
|
||||
val lifecycleGuardian = appLifecycle.guardian
|
||||
val lifecycleGuardian = appLifecycle.asInstanceOf[ActorSystemImpl].guardian
|
||||
val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
|
||||
|
||||
val supervisorSet = receiveWhile(messages = 2) {
|
||||
|
|
@ -150,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
expectNoMsg(Duration.Zero)
|
||||
assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)")
|
||||
|
||||
val actor = new TestActorRef[TestLogActor](app, Props[TestLogActor], supervisor, "none")
|
||||
val actor = TestActorRef[TestLogActor](Props[TestLogActor], supervisor, "none")
|
||||
|
||||
val set = receiveWhile(messages = 2) {
|
||||
case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" ⇒ 1
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import akka.testkit.AkkaSpec
|
|||
class RestartStrategySpec extends AkkaSpec {
|
||||
|
||||
override def atStartup {
|
||||
app.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
|
||||
system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
|
||||
}
|
||||
|
||||
object Ping
|
||||
|
|
|
|||
|
|
@ -28,14 +28,14 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
def receive = { case Tick ⇒ countDownLatch.countDown() }
|
||||
})
|
||||
// run every 50 millisec
|
||||
collectCancellable(app.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(system.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS))
|
||||
|
||||
// after max 1 second it should be executed at least the 3 times already
|
||||
assert(countDownLatch.await(1, TimeUnit.SECONDS))
|
||||
|
||||
val countDownLatch2 = new CountDownLatch(3)
|
||||
|
||||
collectCancellable(app.scheduler.schedule(() ⇒ countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(system.scheduler.schedule(() ⇒ countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS))
|
||||
|
||||
// after max 1 second it should be executed at least the 3 times already
|
||||
assert(countDownLatch2.await(2, TimeUnit.SECONDS))
|
||||
|
|
@ -49,8 +49,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
})
|
||||
|
||||
// run every 50 millisec
|
||||
collectCancellable(app.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(app.scheduler.scheduleOnce(() ⇒ countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(system.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(system.scheduler.scheduleOnce(() ⇒ countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
|
||||
|
||||
// after 1 second the wait should fail
|
||||
assert(countDownLatch.await(2, TimeUnit.SECONDS) == false)
|
||||
|
|
@ -68,10 +68,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
// val actor = actorOf(new Actor {
|
||||
// def receive = { case Ping ⇒ ticks.countDown }
|
||||
// })
|
||||
// val numActors = app.registry.local.actors.length
|
||||
// val numActors = system.registry.local.actors.length
|
||||
// (1 to 1000).foreach(_ ⇒ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS)))
|
||||
// assert(ticks.await(10, TimeUnit.SECONDS))
|
||||
// assert(app.registry.local.actors.length === numActors)
|
||||
// assert(system.registry.local.actors.length === numActors)
|
||||
// }
|
||||
|
||||
/**
|
||||
|
|
@ -86,7 +86,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
})
|
||||
|
||||
(1 to 10).foreach { i ⇒
|
||||
val timeout = collectCancellable(app.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS))
|
||||
val timeout = collectCancellable(system.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS))
|
||||
timeout.cancel()
|
||||
}
|
||||
|
||||
|
|
@ -114,10 +114,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
})
|
||||
val actor = (supervisor ? props).as[ActorRef].get
|
||||
|
||||
collectCancellable(app.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(system.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS))
|
||||
// appx 2 pings before crash
|
||||
EventFilter[Exception]("CRASH", occurrences = 1) intercept {
|
||||
collectCancellable(app.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS))
|
||||
collectCancellable(system.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS))
|
||||
}
|
||||
|
||||
assert(restartLatch.tryAwait(2, TimeUnit.SECONDS))
|
||||
|
|
|
|||
|
|
@ -27,13 +27,13 @@ class SupervisorMiscSpec extends AkkaSpec {
|
|||
}
|
||||
})
|
||||
|
||||
val actor1 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
|
||||
val actor1 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
|
||||
|
||||
val actor2 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
|
||||
val actor2 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
|
||||
|
||||
val actor3 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newDispatcher("test").build)).as[ActorRef].get
|
||||
val actor3 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newDispatcher("test").build)).as[ActorRef].get
|
||||
|
||||
val actor4 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
|
||||
val actor4 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
|
||||
|
||||
actor1 ! Kill
|
||||
actor2 ! Kill
|
||||
|
|
|
|||
|
|
@ -121,7 +121,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
|
|||
}
|
||||
|
||||
override def atStartup() {
|
||||
app.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage)))
|
||||
system.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage)))
|
||||
}
|
||||
|
||||
override def beforeEach() = {
|
||||
|
|
|
|||
|
|
@ -147,18 +147,18 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
newFooBar(Props().withTimeout(Timeout(d)))
|
||||
|
||||
def newFooBar(props: Props): Foo =
|
||||
app.typedActorOf(classOf[Foo], classOf[Bar], props)
|
||||
system.typedActorOf(classOf[Foo], classOf[Bar], props)
|
||||
|
||||
def newStacked(props: Props = Props().withTimeout(Timeout(2000))): Stacked =
|
||||
app.typedActorOf(classOf[Stacked], classOf[StackedImpl], props)
|
||||
system.typedActorOf(classOf[Stacked], classOf[StackedImpl], props)
|
||||
|
||||
def mustStop(typedActor: AnyRef) = app.typedActor.stop(typedActor) must be(true)
|
||||
def mustStop(typedActor: AnyRef) = system.typedActor.stop(typedActor) must be(true)
|
||||
|
||||
"TypedActors" must {
|
||||
|
||||
"be able to instantiate" in {
|
||||
val t = newFooBar
|
||||
app.typedActor.isTypedActor(t) must be(true)
|
||||
system.typedActor.isTypedActor(t) must be(true)
|
||||
mustStop(t)
|
||||
}
|
||||
|
||||
|
|
@ -168,7 +168,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
}
|
||||
|
||||
"not stop non-started ones" in {
|
||||
app.typedActor.stop(null) must be(false)
|
||||
system.typedActor.stop(null) must be(false)
|
||||
}
|
||||
|
||||
"throw an IllegalStateExcpetion when TypedActor.self is called in the wrong scope" in {
|
||||
|
|
@ -187,7 +187,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
"be able to call toString" in {
|
||||
val t = newFooBar
|
||||
t.toString must be(app.typedActor.getActorRefFor(t).toString)
|
||||
t.toString must be(system.typedActor.getActorRefFor(t).toString)
|
||||
mustStop(t)
|
||||
}
|
||||
|
||||
|
|
@ -200,7 +200,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
"be able to call hashCode" in {
|
||||
val t = newFooBar
|
||||
t.hashCode must be(app.typedActor.getActorRefFor(t).hashCode)
|
||||
t.hashCode must be(system.typedActor.getActorRefFor(t).hashCode)
|
||||
mustStop(t)
|
||||
}
|
||||
|
||||
|
|
@ -295,7 +295,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
}
|
||||
|
||||
"be able to support implementation only typed actors" in {
|
||||
val t = app.typedActorOf[Foo, Bar](Props())
|
||||
val t = system.typedActorOf[Foo, Bar](Props())
|
||||
val f = t.futurePigdog(200)
|
||||
val f2 = t.futurePigdog(0)
|
||||
f2.isCompleted must be(false)
|
||||
|
|
@ -305,7 +305,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
}
|
||||
|
||||
"be able to support implementation only typed actors with complex interfaces" in {
|
||||
val t = app.typedActorOf[Stackable1 with Stackable2, StackedImpl]()
|
||||
val t = system.typedActorOf[Stackable1 with Stackable2, StackedImpl]()
|
||||
t.stackable1 must be("foo")
|
||||
t.stackable2 must be("bar")
|
||||
mustStop(t)
|
||||
|
|
@ -314,7 +314,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
"be able to use work-stealing dispatcher" in {
|
||||
val props = Props(
|
||||
timeout = Timeout(6600),
|
||||
dispatcher = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
|
||||
dispatcher = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(60)
|
||||
.setMaxPoolSize(60)
|
||||
|
|
@ -332,7 +332,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
"be able to serialize and deserialize invocations" in {
|
||||
import java.io._
|
||||
val m = TypedActor.MethodCall(app, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
|
||||
val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
|
||||
val baos = new ByteArrayOutputStream(8192 * 4)
|
||||
val out = new ObjectOutputStream(baos)
|
||||
|
||||
|
|
@ -341,7 +341,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
|
||||
|
||||
mNew.method must be(m.method)
|
||||
|
|
@ -351,7 +351,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
"be able to serialize and deserialize invocations' parameters" in {
|
||||
import java.io._
|
||||
val someFoo: Foo = new Bar
|
||||
val m = TypedActor.MethodCall(app, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
|
||||
val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
|
||||
val baos = new ByteArrayOutputStream(8192 * 4)
|
||||
val out = new ObjectOutputStream(baos)
|
||||
|
||||
|
|
@ -360,7 +360,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
|
||||
|
||||
mNew.method must be(m.method)
|
||||
|
|
|
|||
|
|
@ -141,13 +141,13 @@ object ActorModelSpec {
|
|||
}
|
||||
|
||||
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
|
||||
stops: Long = dispatcher.stops.get())(implicit app: ActorSystem) {
|
||||
stops: Long = dispatcher.stops.get())(implicit system: ActorSystem) {
|
||||
val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
|
||||
try {
|
||||
await(deadline)(stops == dispatcher.stops.get)
|
||||
} catch {
|
||||
case e ⇒
|
||||
app.eventStream.publish(Error(e, dispatcher, "actual: stops=" + dispatcher.stops.get +
|
||||
system.eventStream.publish(Error(e, dispatcher, "actual: stops=" + dispatcher.stops.get +
|
||||
" required: stops=" + stops))
|
||||
throw e
|
||||
}
|
||||
|
|
@ -173,7 +173,7 @@ object ActorModelSpec {
|
|||
unregisters: Long = 0,
|
||||
msgsReceived: Long = 0,
|
||||
msgsProcessed: Long = 0,
|
||||
restarts: Long = 0)(implicit app: ActorSystem) {
|
||||
restarts: Long = 0)(implicit system: ActorSystem) {
|
||||
assertRef(actorRef, dispatcher)(
|
||||
suspensions,
|
||||
resumes,
|
||||
|
|
@ -191,7 +191,7 @@ object ActorModelSpec {
|
|||
unregisters: Long = statsFor(actorRef).unregisters.get(),
|
||||
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
|
||||
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
|
||||
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: ActorSystem) {
|
||||
restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
|
||||
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
|
||||
val deadline = System.currentTimeMillis + 1000
|
||||
try {
|
||||
|
|
@ -204,7 +204,7 @@ object ActorModelSpec {
|
|||
await(deadline)(stats.restarts.get() == restarts)
|
||||
} catch {
|
||||
case e ⇒
|
||||
app.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
|
||||
system.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
|
||||
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
|
||||
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
|
||||
throw e
|
||||
|
|
@ -227,7 +227,7 @@ abstract class ActorModelSpec extends AkkaSpec {
|
|||
|
||||
import ActorModelSpec._
|
||||
|
||||
def newTestActor(dispatcher: MessageDispatcher) = app.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||
def newTestActor(dispatcher: MessageDispatcher) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||
|
||||
protected def newInterceptedDispatcher: MessageDispatcherInterceptor
|
||||
protected def dispatcherType: String
|
||||
|
|
@ -310,7 +310,7 @@ abstract class ActorModelSpec extends AkkaSpec {
|
|||
try {
|
||||
f
|
||||
} catch {
|
||||
case e ⇒ app.eventStream.publish(Error(e, this, "error in spawned thread"))
|
||||
case e ⇒ system.eventStream.publish(Error(e, this, "error in spawned thread"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -407,10 +407,10 @@ class DispatcherModelSpec extends ActorModelSpec {
|
|||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(app, "foo", app.AkkaConfig.DispatcherThroughput,
|
||||
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
|
||||
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, "foo", system.settings.DispatcherThroughput,
|
||||
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
|
||||
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
|
||||
def dispatcherType = "Dispatcher"
|
||||
|
||||
|
|
@ -444,14 +444,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
|
|||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(app, "foo", 1, // TODO check why 1 here? (came from old test)
|
||||
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
|
||||
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
new BalancingDispatcher(system.dispatcherFactory.prerequisites, "foo", 1, // TODO check why 1 here? (came from old test)
|
||||
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
|
||||
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
|
||||
def dispatcherType = "Balancing Dispatcher"
|
||||
|
||||
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = app.dispatcher
|
||||
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = system.dispatcher
|
||||
|
||||
"A " + dispatcherType must {
|
||||
"process messages in parallel" in {
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import akka.testkit.AkkaSpec
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class BalancingDispatcherSpec extends AkkaSpec {
|
||||
|
||||
def newWorkStealer() = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build
|
||||
def newWorkStealer() = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build
|
||||
|
||||
val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer()
|
||||
|
||||
|
|
|
|||
|
|
@ -33,22 +33,22 @@ class DispatcherActorSpec extends AkkaSpec {
|
|||
"A Dispatcher and an Actor" must {
|
||||
|
||||
"support tell" in {
|
||||
val actor = actorOf(Props[OneWayTestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build))
|
||||
val actor = actorOf(Props[OneWayTestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
|
||||
val result = actor ! "OneWay"
|
||||
assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
|
||||
actor.stop()
|
||||
}
|
||||
|
||||
"support ask/reply" in {
|
||||
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build))
|
||||
val actor = actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
|
||||
val result = (actor ? "Hello").as[String]
|
||||
assert("World" === result.get)
|
||||
actor.stop()
|
||||
}
|
||||
|
||||
"respect the throughput setting" in {
|
||||
val throughputDispatcher = app.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType).
|
||||
val throughputDispatcher = system.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 101, 0, system.dispatcherFactory.MailboxType).
|
||||
setCorePoolSize(1).
|
||||
build
|
||||
|
||||
|
|
@ -76,8 +76,8 @@ class DispatcherActorSpec extends AkkaSpec {
|
|||
|
||||
"respect throughput deadline" in {
|
||||
val deadlineMs = 100
|
||||
val throughputDispatcher = app.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 2, deadlineMs, app.dispatcherFactory.MailboxType).
|
||||
val throughputDispatcher = system.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 2, deadlineMs, system.dispatcherFactory.MailboxType).
|
||||
setCorePoolSize(1).
|
||||
build
|
||||
val works = new AtomicBoolean(true)
|
||||
|
|
|
|||
|
|
@ -12,7 +12,8 @@ import akka.config.Configuration
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class DispatchersSpec extends AkkaSpec {
|
||||
|
||||
import app.dispatcherFactory._
|
||||
val df = system.dispatcherFactory
|
||||
import df._
|
||||
|
||||
val tipe = "type"
|
||||
val keepalivems = "keep-alive-time"
|
||||
|
|
|
|||
|
|
@ -27,14 +27,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
|
||||
"support tell" in {
|
||||
var oneWay = new CountDownLatch(1)
|
||||
val actor = actorOf(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test")))
|
||||
val actor = actorOf(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
|
||||
val result = actor ! "OneWay"
|
||||
assert(oneWay.await(1, TimeUnit.SECONDS))
|
||||
actor.stop()
|
||||
}
|
||||
|
||||
"support ask/reply" in {
|
||||
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test")))
|
||||
val actor = actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
|
||||
val result = (actor ? "Hello").as[String]
|
||||
assert("World" === result.get)
|
||||
actor.stop()
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ class ConfigSpec extends AkkaSpec(ActorSystem("ConfigSpec", Configuration.fromFi
|
|||
"The default configuration file (i.e. akka-reference.conf)" must {
|
||||
"contain all configuration properties for akka-actor that are used in code with their correct defaults" in {
|
||||
|
||||
val config = app.config
|
||||
val config = system.settings.config
|
||||
import config._
|
||||
|
||||
getList("akka.boot") must equal(Nil)
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class Future2ActorSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"support reply via sender" in {
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "do" ⇒ Future(31) pipeTo context.sender
|
||||
case "ex" ⇒ Future(throw new AssertionError) pipeTo context.sender
|
||||
|
|
|
|||
|
|
@ -80,7 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
|
|||
result
|
||||
}
|
||||
|
||||
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, app.deadLetters)
|
||||
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)
|
||||
|
||||
def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
|
||||
q must not be null
|
||||
|
|
|
|||
|
|
@ -18,12 +18,12 @@ class PriorityDispatcherSpec extends AkkaSpec {
|
|||
testOrdering(BoundedPriorityMailbox(PriorityGenerator({
|
||||
case i: Int ⇒ i //Reverse order
|
||||
case 'Result ⇒ Int.MaxValue
|
||||
}: Any ⇒ Int), 1000, app.AkkaConfig.MailboxPushTimeout))
|
||||
}: Any ⇒ Int), 1000, system.settings.MailboxPushTimeout))
|
||||
}
|
||||
}
|
||||
|
||||
def testOrdering(mboxType: MailboxType) {
|
||||
val dispatcher = app.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
|
||||
val dispatcher = system.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
|
||||
|
||||
val actor = actorOf(Props(new Actor {
|
||||
var acc: List[Int] = Nil
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.event
|
|||
import akka.testkit.AkkaSpec
|
||||
import akka.config.Configuration
|
||||
import akka.util.duration._
|
||||
import akka.actor.{ Actor, ActorRef }
|
||||
import akka.actor.{ Actor, ActorRef, ActorSystemImpl }
|
||||
|
||||
object EventStreamSpec {
|
||||
case class M(i: Int)
|
||||
|
|
@ -14,7 +14,7 @@ object EventStreamSpec {
|
|||
case class SetTarget(ref: ActorRef)
|
||||
|
||||
class MyLog extends Actor {
|
||||
var dst: ActorRef = app.deadLetters
|
||||
var dst: ActorRef = system.deadLetters
|
||||
def receive = {
|
||||
case Logging.InitializeLogger(bus) ⇒ bus.subscribe(context.self, classOf[SetTarget]); sender ! Logging.LoggerInitialized
|
||||
case SetTarget(ref) ⇒ dst = ref; dst ! "OK"
|
||||
|
|
@ -36,11 +36,13 @@ class EventStreamSpec extends AkkaSpec(Configuration(
|
|||
|
||||
import EventStreamSpec._
|
||||
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
|
||||
"An EventStream" must {
|
||||
|
||||
"manage subscriptions" in {
|
||||
val bus = new EventStream(true)
|
||||
bus.start(app)
|
||||
bus.start(impl)
|
||||
bus.subscribe(testActor, classOf[M])
|
||||
bus.publish(M(42))
|
||||
within(1 second) {
|
||||
|
|
@ -53,8 +55,8 @@ class EventStreamSpec extends AkkaSpec(Configuration(
|
|||
|
||||
"manage log levels" in {
|
||||
val bus = new EventStream(false)
|
||||
bus.start(app)
|
||||
bus.startDefaultLoggers(app, app.AkkaConfig)
|
||||
bus.start(impl)
|
||||
bus.startDefaultLoggers(impl)
|
||||
bus.publish(SetTarget(testActor))
|
||||
expectMsg("OK")
|
||||
within(2 seconds) {
|
||||
|
|
@ -75,7 +77,7 @@ class EventStreamSpec extends AkkaSpec(Configuration(
|
|||
val b2 = new B2
|
||||
val c = new C
|
||||
val bus = new EventStream(false)
|
||||
bus.start(app)
|
||||
bus.start(impl)
|
||||
within(2 seconds) {
|
||||
bus.subscribe(testActor, classOf[B2]) === true
|
||||
bus.publish(c)
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic
|
|||
class TellLatencyPerformanceSpec extends PerformanceSpec {
|
||||
import TellLatencyPerformanceSpec._
|
||||
|
||||
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(8)
|
||||
.build
|
||||
|
|
@ -62,13 +62,13 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
|
|||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val clients = (for (i ← 0 until numberOfClients) yield {
|
||||
val destination = app.actorOf[Destination]
|
||||
val w4 = app.actorOf(new Waypoint(destination))
|
||||
val w3 = app.actorOf(new Waypoint(w4))
|
||||
val w2 = app.actorOf(new Waypoint(w3))
|
||||
val w1 = app.actorOf(new Waypoint(w2))
|
||||
val destination = system.actorOf[Destination]
|
||||
val w4 = system.actorOf(new Waypoint(destination))
|
||||
val w3 = system.actorOf(new Waypoint(w4))
|
||||
val w2 = system.actorOf(new Waypoint(w3))
|
||||
val w1 = system.actorOf(new Waypoint(w2))
|
||||
Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher)
|
||||
}).toList.map(app.actorOf(_))
|
||||
}).toList.map(system.actorOf(_))
|
||||
|
||||
val start = System.nanoTime
|
||||
clients.foreach(_ ! Run)
|
||||
|
|
|
|||
|
|
@ -12,8 +12,9 @@ import java.util.concurrent.ThreadPoolExecutor.AbortPolicy
|
|||
class TellThroughputPerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughputPerformanceSpec._
|
||||
|
||||
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(app, name, 5,
|
||||
0, UnboundedMailbox(), config, 60000), ThreadPoolConfig(app))
|
||||
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
|
||||
0, UnboundedMailbox(), config, 60000), ThreadPoolConfig())
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
|
@ -71,9 +72,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
|
|||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
yield app.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
|
||||
val clients = for (dest ← destinations)
|
||||
yield app.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
|
||||
|
||||
val start = System.nanoTime
|
||||
clients.foreach(_ ! Run)
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class TradingLatencyPerformanceSpec extends PerformanceSpec {
|
||||
|
||||
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
|
@ -38,7 +38,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
|
|||
override def beforeEach() {
|
||||
super.beforeEach()
|
||||
stat = new SynchronizedDescriptiveStatistics
|
||||
tradingSystem = new AkkaTradingSystem(app)
|
||||
tradingSystem = new AkkaTradingSystem(system)
|
||||
tradingSystem.start()
|
||||
TotalTradeCounter.reset()
|
||||
stat = new SynchronizedDescriptiveStatistics
|
||||
|
|
@ -99,7 +99,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
|
|||
val clients = (for (i ← 0 until numberOfClients) yield {
|
||||
val receiver = receivers(i % receivers.size)
|
||||
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher)
|
||||
app.actorOf(props)
|
||||
system.actorOf(props)
|
||||
})
|
||||
|
||||
clients.foreach(_ ! "run")
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ trait TradingSystem {
|
|||
case class MatchingEngineInfo(primary: ME, standby: Option[ME], orderbooks: List[Orderbook])
|
||||
}
|
||||
|
||||
class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
|
||||
class AkkaTradingSystem(val system: ActorSystem) extends TradingSystem {
|
||||
type ME = ActorRef
|
||||
type OR = ActorRef
|
||||
|
||||
|
|
@ -70,8 +70,8 @@ class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
|
|||
|
||||
def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) =
|
||||
meDispatcher match {
|
||||
case Some(d) ⇒ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d))
|
||||
case _ ⇒ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)))
|
||||
case Some(d) ⇒ system.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d))
|
||||
case _ ⇒ system.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)))
|
||||
}
|
||||
|
||||
override def createOrderReceivers: List[ActorRef] = {
|
||||
|
|
@ -91,8 +91,8 @@ class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem {
|
|||
}
|
||||
|
||||
def createOrderReceiver() = orDispatcher match {
|
||||
case Some(d) ⇒ app.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d))
|
||||
case _ ⇒ app.actorOf(Props(new AkkaOrderReceiver()))
|
||||
case Some(d) ⇒ system.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d))
|
||||
case _ ⇒ system.actorOf(Props(new AkkaOrderReceiver()))
|
||||
}
|
||||
|
||||
override def start() {
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class TradingThroughputPerformanceSpec extends PerformanceSpec {
|
||||
|
||||
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
|
@ -30,7 +30,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
|
|||
|
||||
override def beforeEach() {
|
||||
super.beforeEach()
|
||||
tradingSystem = new AkkaTradingSystem(app)
|
||||
tradingSystem = new AkkaTradingSystem(system)
|
||||
tradingSystem.start()
|
||||
TotalTradeCounter.reset()
|
||||
}
|
||||
|
|
@ -92,7 +92,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
|
|||
val clients = (for (i ← 0 until numberOfClients) yield {
|
||||
val receiver = receivers(i % receivers.size)
|
||||
val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher)
|
||||
app.actorOf(props)
|
||||
system.actorOf(props)
|
||||
})
|
||||
|
||||
clients.foreach(_ ! "run")
|
||||
|
|
|
|||
|
|
@ -11,8 +11,6 @@ import akka.actor.ActorSystem
|
|||
|
||||
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
|
||||
|
||||
def app: ActorSystem
|
||||
|
||||
def isBenchmark() = System.getProperty("benchmark") == "true"
|
||||
|
||||
def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
|
||||
|
|
@ -29,7 +27,7 @@ trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
}
|
||||
|
||||
val resultRepository = BenchResultRepository()
|
||||
lazy val report = new Report(app, resultRepository, compareResultWith)
|
||||
lazy val report = new Report(system, resultRepository, compareResultWith)
|
||||
|
||||
/**
|
||||
* To compare two tests with each other you can override this method, in
|
||||
|
|
|
|||
|
|
@ -10,12 +10,12 @@ import akka.event.Logging
|
|||
import scala.collection.immutable.TreeMap
|
||||
|
||||
class Report(
|
||||
app: ActorSystem,
|
||||
system: ActorSystem,
|
||||
resultRepository: BenchResultRepository,
|
||||
compareResultWith: Option[String] = None) {
|
||||
|
||||
private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
val dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
|
||||
val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
|
||||
|
|
@ -221,11 +221,11 @@ class Report(
|
|||
sb.append("Args:\n ").append(args)
|
||||
sb.append("\n")
|
||||
|
||||
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
|
||||
sb.append("Akka version: ").append(system.settings.ConfigVersion)
|
||||
sb.append("\n")
|
||||
sb.append("Akka config:")
|
||||
for (key ← app.config.keys) {
|
||||
sb.append("\n ").append(key).append("=").append(app.config(key))
|
||||
for (key ← system.settings.config.keys) {
|
||||
sb.append("\n ").append(key).append("=").append(system.settings.config(key))
|
||||
}
|
||||
|
||||
sb.toString
|
||||
|
|
|
|||
|
|
@ -29,7 +29,7 @@ class TypedActorPoolSpec extends AkkaSpec {
|
|||
import ActorPoolSpec._
|
||||
"Actor Pool (2)" must {
|
||||
"support typed actors" in {
|
||||
val pool = app.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
|
||||
val pool = system.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
|
||||
def lowerBound = 1
|
||||
def upperBound = 5
|
||||
def pressureThreshold = 1
|
||||
|
|
@ -38,7 +38,7 @@ class TypedActorPoolSpec extends AkkaSpec {
|
|||
def rampupRate = 0.1
|
||||
def backoffRate = 0.50
|
||||
def backoffThreshold = 0.50
|
||||
def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
|
||||
def instance(p: Props) = system.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
|
||||
def receive = _route
|
||||
}, Props().withTimeout(10 seconds).withFaultHandler(faultHandler))
|
||||
|
||||
|
|
@ -47,7 +47,7 @@ class TypedActorPoolSpec extends AkkaSpec {
|
|||
for ((i, r) ← results)
|
||||
r.get must equal(i * i)
|
||||
|
||||
app.typedActor.stop(pool)
|
||||
system.typedActor.stop(pool)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,12 +11,14 @@ import akka.routing.Routing.Broadcast
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
||||
|
||||
val deployer = system.asInstanceOf[ActorSystemImpl].provider.deployer
|
||||
|
||||
"round robin router" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val path = app / "round-robin-0"
|
||||
val path = system / "round-robin-0"
|
||||
|
||||
app.provider.deployer.deploy(
|
||||
deployer.deploy(
|
||||
Deploy(
|
||||
path.toString,
|
||||
None,
|
||||
|
|
@ -27,7 +29,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
val helloLatch = new CountDownLatch(5)
|
||||
val stopLatch = new CountDownLatch(5)
|
||||
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
|
@ -49,9 +51,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val path = app / "round-robin-1"
|
||||
val path = system / "round-robin-1"
|
||||
|
||||
app.provider.deployer.deploy(
|
||||
deployer.deploy(
|
||||
Deploy(
|
||||
path.toString,
|
||||
None,
|
||||
|
|
@ -69,7 +71,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
replies = replies + (i -> 0)
|
||||
}
|
||||
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
|
|
@ -93,9 +95,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val path = app / "round-robin-2"
|
||||
val path = system / "round-robin-2"
|
||||
|
||||
app.provider.deployer.deploy(
|
||||
deployer.deploy(
|
||||
Deploy(
|
||||
path.toString,
|
||||
None,
|
||||
|
|
@ -106,7 +108,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
val helloLatch = new CountDownLatch(5)
|
||||
val stopLatch = new CountDownLatch(5)
|
||||
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
|
@ -127,9 +129,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
"random router" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val path = app / "random-0"
|
||||
val path = system / "random-0"
|
||||
|
||||
app.provider.deployer.deploy(
|
||||
deployer.deploy(
|
||||
Deploy(
|
||||
path.toString,
|
||||
None,
|
||||
|
|
@ -139,7 +141,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
|
||||
val stopLatch = new CountDownLatch(7)
|
||||
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ {}
|
||||
}
|
||||
|
|
@ -160,9 +162,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"deliver messages in a random fashion" in {
|
||||
val path = app / "random-1"
|
||||
val path = system / "random-1"
|
||||
|
||||
app.provider.deployer.deploy(
|
||||
deployer.deploy(
|
||||
Deploy(
|
||||
path.toString,
|
||||
None,
|
||||
|
|
@ -180,7 +182,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
replies = replies + (i -> 0)
|
||||
}
|
||||
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
|
|
@ -204,9 +206,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val path = app / "random-2"
|
||||
val path = system / "random-2"
|
||||
|
||||
app.provider.deployer.deploy(
|
||||
deployer.deploy(
|
||||
Deploy(
|
||||
path.toString,
|
||||
None,
|
||||
|
|
@ -217,7 +219,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
|
|||
val helloLatch = new CountDownLatch(6)
|
||||
val stopLatch = new CountDownLatch(6)
|
||||
|
||||
val actor = app.actorOf(Props(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,6 +22,8 @@ object RoutingSpec {
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RoutingSpec extends AkkaSpec {
|
||||
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
|
||||
import akka.routing.RoutingSpec._
|
||||
|
||||
"direct router" must {
|
||||
|
|
@ -29,7 +31,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
val actor1 = actorOf[TestActor]
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
actor.isShutdown must be(false)
|
||||
}
|
||||
|
||||
|
|
@ -45,7 +47,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val routedActor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val routedActor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
routedActor ! "hello"
|
||||
routedActor ! "end"
|
||||
|
||||
|
|
@ -66,7 +68,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor ! Broadcast(1)
|
||||
actor ! "end"
|
||||
|
|
@ -83,7 +85,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
val actor1 = actorOf[TestActor]
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
actor.isShutdown must be(false)
|
||||
}
|
||||
|
||||
|
|
@ -113,7 +115,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
//create the routed actor.
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(connections))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
//send messages to the actor.
|
||||
for (i ← 0 until iterationCount) {
|
||||
|
|
@ -152,7 +154,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor ! Broadcast(1)
|
||||
actor ! Broadcast("end")
|
||||
|
|
@ -175,7 +177,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
intercept[RoutingException] { actor ? Broadcast(1) }
|
||||
|
||||
|
|
@ -192,7 +194,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
val actor1 = actorOf[TestActor]
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
actor.isShutdown must be(false)
|
||||
}
|
||||
|
||||
|
|
@ -216,7 +218,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor ! Broadcast(1)
|
||||
actor ! Broadcast("end")
|
||||
|
|
@ -239,7 +241,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
try {
|
||||
actor ? Broadcast(1)
|
||||
|
|
@ -262,7 +264,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))))
|
||||
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor ! Broadcast(Stop(Some(0)))
|
||||
|
||||
|
|
@ -277,7 +279,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))))
|
||||
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor ! Broadcast(Stop())
|
||||
|
||||
|
|
@ -293,7 +295,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1))))
|
||||
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
(actor ? Broadcast("Hi!")).get.asInstanceOf[Int] must be(0)
|
||||
|
||||
|
|
@ -302,14 +304,14 @@ class RoutingSpec extends AkkaSpec {
|
|||
"return the first response from connections, when some of them failed to reply" in {
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1))))
|
||||
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
|
||||
}
|
||||
|
||||
"be started when constructed" in {
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0))))
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor.isShutdown must be(false)
|
||||
}
|
||||
|
|
@ -324,7 +326,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
for (i ← 0 until connectionCount) {
|
||||
counters = counters :+ new AtomicInteger()
|
||||
|
||||
val connection = app.actorOf(new Actor {
|
||||
val connection = system.actorOf(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counters.get(i).get.addAndGet(msg)
|
||||
|
|
@ -335,7 +337,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(connections))
|
||||
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
|
|
@ -357,7 +359,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = app.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
|
|
@ -365,7 +367,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val connection2 = app.actorOf(new Actor {
|
||||
val connection2 = system.actorOf(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
|
|
@ -374,7 +376,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
|
||||
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
||||
actor ! Broadcast(1)
|
||||
actor ! Broadcast("end")
|
||||
|
|
@ -387,7 +389,7 @@ class RoutingSpec extends AkkaSpec {
|
|||
|
||||
case class Stop(id: Option[Int] = None)
|
||||
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = app.actorOf(new Actor {
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(new Actor {
|
||||
def receive = {
|
||||
case Stop(None) ⇒ self.stop()
|
||||
case Stop(Some(_id)) if (_id == id) ⇒ self.stop()
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.serialization
|
|||
import akka.serialization.Serialization._
|
||||
import scala.reflect._
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
|
||||
import akka.actor.DeadLetterActorRef
|
||||
|
||||
|
|
@ -24,7 +24,8 @@ object SerializeSpec {
|
|||
class SerializeSpec extends AkkaSpec {
|
||||
import SerializeSpec._
|
||||
|
||||
import app.serialization._
|
||||
val ser = system.serialization
|
||||
import ser._
|
||||
|
||||
"Serialization" must {
|
||||
|
||||
|
|
@ -68,13 +69,13 @@ class SerializeSpec extends AkkaSpec {
|
|||
"serialize DeadLetterActorRef" in {
|
||||
val outbuf = new ByteArrayOutputStream()
|
||||
val out = new ObjectOutputStream(outbuf)
|
||||
val a = new ActorSystem()
|
||||
val a = ActorSystem()
|
||||
out.writeObject(a.deadLetters)
|
||||
out.flush()
|
||||
out.close()
|
||||
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
|
||||
Serialization.app.withValue(a) {
|
||||
Serialization.system.withValue(a.asInstanceOf[ActorSystemImpl]) {
|
||||
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
|
||||
(deadLetters eq a.deadLetters) must be(true)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import org.junit.{ After, Test }
|
|||
class CallingThreadDispatcherModelSpec extends ActorModelSpec {
|
||||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = new CallingThreadDispatcher(app, "test") with MessageDispatcherInterceptor
|
||||
def newInterceptedDispatcher = new CallingThreadDispatcher(system.dispatcherFactory.prerequisites, "test") with MessageDispatcherInterceptor
|
||||
def dispatcherType = "Calling Thread Dispatcher"
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -150,11 +150,11 @@ object Timeout {
|
|||
implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
|
||||
implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
|
||||
implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
|
||||
implicit def defaultTimeout(implicit app: ActorSystem) = app.AkkaConfig.ActorTimeout
|
||||
implicit def defaultTimeout(implicit system: ActorSystem) = system.settings.ActorTimeout
|
||||
}
|
||||
|
||||
trait ActorLogging { this: Actor ⇒
|
||||
val log = akka.event.Logging(app.eventStream, context.self)
|
||||
val log = akka.event.Logging(system.eventStream, context.self)
|
||||
}
|
||||
|
||||
object Actor {
|
||||
|
|
@ -164,17 +164,17 @@ object Actor {
|
|||
/**
|
||||
* This decorator adds invocation logging to a Receive function.
|
||||
*/
|
||||
class LoggingReceive(source: AnyRef, r: Receive)(implicit app: ActorSystem) extends Receive {
|
||||
class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive {
|
||||
def isDefinedAt(o: Any) = {
|
||||
val handled = r.isDefinedAt(o)
|
||||
app.eventStream.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
|
||||
system.eventStream.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
|
||||
handled
|
||||
}
|
||||
def apply(o: Any): Unit = r(o)
|
||||
}
|
||||
|
||||
object LoggingReceive {
|
||||
def apply(source: AnyRef, r: Receive)(implicit app: ActorSystem): Receive = r match {
|
||||
def apply(source: AnyRef, r: Receive)(implicit system: ActorSystem): Receive = r match {
|
||||
case _: LoggingReceive ⇒ r
|
||||
case _ ⇒ new LoggingReceive(source, r)
|
||||
}
|
||||
|
|
@ -229,12 +229,12 @@ trait Actor {
|
|||
c
|
||||
}
|
||||
|
||||
implicit def app = context.app
|
||||
implicit def system = context.system
|
||||
|
||||
/**
|
||||
* The default timeout, based on the config setting 'akka.actor.timeout'
|
||||
*/
|
||||
implicit def defaultTimeout = app.AkkaConfig.ActorTimeout
|
||||
implicit def defaultTimeout = system.settings.ActorTimeout
|
||||
|
||||
/**
|
||||
* Wrap a Receive partial function in a logging enclosure, which sends a
|
||||
|
|
@ -250,7 +250,7 @@ trait Actor {
|
|||
* This method does NOT modify the given Receive unless
|
||||
* akka.actor.debug.receive is set within akka.conf.
|
||||
*/
|
||||
def loggable(self: AnyRef)(r: Receive): Receive = if (app.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
|
||||
def loggable(self: AnyRef)(r: Receive): Receive = if (system.settings.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
|
||||
|
||||
/**
|
||||
* The 'self' field holds the ActorRef for this actor.
|
||||
|
|
|
|||
|
|
@ -45,7 +45,7 @@ trait ActorContext extends ActorRefFactory with TypedActorFactory {
|
|||
|
||||
def handleChildTerminated(child: ActorRef): Unit
|
||||
|
||||
def app: ActorSystem
|
||||
def system: ActorSystem
|
||||
|
||||
def parent: ActorRef
|
||||
}
|
||||
|
|
@ -63,7 +63,7 @@ private[akka] object ActorCell {
|
|||
//vars don't need volatile since it's protected with the mailbox status
|
||||
//Make sure that they are not read/written outside of a message processing (systemInvoke/invoke)
|
||||
private[akka] class ActorCell(
|
||||
val app: ActorSystem,
|
||||
val system: ActorSystemImpl,
|
||||
val self: ActorRef with ScalaActorRef,
|
||||
val props: Props,
|
||||
val parent: ActorRef,
|
||||
|
|
@ -72,11 +72,13 @@ private[akka] class ActorCell(
|
|||
|
||||
import ActorCell._
|
||||
|
||||
def systemImpl = system
|
||||
|
||||
protected final def guardian = self
|
||||
|
||||
protected def typedActor = app.typedActor
|
||||
protected def typedActor = system.typedActor
|
||||
|
||||
final def provider = app.provider
|
||||
final def provider = system.provider
|
||||
|
||||
var futureTimeout: Option[Cancellable] = None
|
||||
|
||||
|
|
@ -91,7 +93,7 @@ private[akka] class ActorCell(
|
|||
var stopping = false
|
||||
|
||||
@inline
|
||||
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher
|
||||
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
|
||||
|
||||
final def isShutdown: Boolean = mailbox.isClosed
|
||||
|
||||
|
|
@ -139,12 +141,12 @@ private[akka] class ActorCell(
|
|||
}
|
||||
|
||||
final def tell(message: Any, sender: ActorRef): Unit =
|
||||
dispatcher.dispatch(this, Envelope(message, if (sender eq null) app.deadLetters else sender))
|
||||
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender))
|
||||
|
||||
final def sender: ActorRef = currentMessage match {
|
||||
case null ⇒ app.deadLetters
|
||||
case null ⇒ system.deadLetters
|
||||
case msg if msg.sender ne null ⇒ msg.sender
|
||||
case _ ⇒ app.deadLetters
|
||||
case _ ⇒ system.deadLetters
|
||||
}
|
||||
|
||||
//This method is in charge of setting up the contextStack and create a new instance of the Actor
|
||||
|
|
@ -172,11 +174,11 @@ private[akka] class ActorCell(
|
|||
actor = created
|
||||
created.preStart()
|
||||
checkReceiveTimeout
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")"))
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "started (" + actor + ")"))
|
||||
} catch {
|
||||
case e ⇒
|
||||
try {
|
||||
app.eventStream.publish(Error(e, self, "error while creating actor"))
|
||||
system.eventStream.publish(Error(e, self, "error while creating actor"))
|
||||
// prevent any further messages to be processed until the actor has been restarted
|
||||
dispatcher.suspend(this)
|
||||
} finally {
|
||||
|
|
@ -186,7 +188,7 @@ private[akka] class ActorCell(
|
|||
|
||||
def recreate(cause: Throwable): Unit = try {
|
||||
val failedActor = actor
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting"))
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "restarting"))
|
||||
val freshActor = newActor()
|
||||
if (failedActor ne null) {
|
||||
val c = currentMessage //One read only plz
|
||||
|
|
@ -200,14 +202,14 @@ private[akka] class ActorCell(
|
|||
}
|
||||
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
|
||||
freshActor.postRestart(cause)
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted"))
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "restarted"))
|
||||
|
||||
dispatcher.resume(this) //FIXME should this be moved down?
|
||||
|
||||
props.faultHandler.handleSupervisorRestarted(cause, self, children)
|
||||
} catch {
|
||||
case e ⇒ try {
|
||||
app.eventStream.publish(Error(e, self, "error while creating actor"))
|
||||
system.eventStream.publish(Error(e, self, "error while creating actor"))
|
||||
// prevent any further messages to be processed until the actor has been restarted
|
||||
dispatcher.suspend(this)
|
||||
} finally {
|
||||
|
|
@ -226,7 +228,7 @@ private[akka] class ActorCell(
|
|||
val c = children
|
||||
if (c.isEmpty) doTerminate()
|
||||
else {
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping"))
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopping"))
|
||||
for (child ← c) child.stop()
|
||||
stopping = true
|
||||
}
|
||||
|
|
@ -237,8 +239,8 @@ private[akka] class ActorCell(
|
|||
if (!stats.contains(child)) {
|
||||
childrenRefs = childrenRefs.updated(child.name, child)
|
||||
childrenStats = childrenStats.updated(child, ChildRestartStats())
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child))
|
||||
} else app.eventStream.publish(Warning(self, "Already supervising " + child))
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now supervising " + child))
|
||||
} else system.eventStream.publish(Warning(self, "Already supervising " + child))
|
||||
}
|
||||
|
||||
try {
|
||||
|
|
@ -252,11 +254,11 @@ private[akka] class ActorCell(
|
|||
case Create() ⇒ create()
|
||||
case Recreate(cause) ⇒ recreate(cause)
|
||||
case Link(subject) ⇒
|
||||
app.deathWatch.subscribe(self, subject)
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject))
|
||||
system.deathWatch.subscribe(self, subject)
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now monitoring " + subject))
|
||||
case Unlink(subject) ⇒
|
||||
app.deathWatch.unsubscribe(self, subject)
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject))
|
||||
system.deathWatch.unsubscribe(self, subject)
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped monitoring " + subject))
|
||||
case Suspend() ⇒ suspend()
|
||||
case Resume() ⇒ resume()
|
||||
case Terminate() ⇒ terminate()
|
||||
|
|
@ -265,7 +267,7 @@ private[akka] class ActorCell(
|
|||
}
|
||||
} catch {
|
||||
case e ⇒ //Should we really catch everything here?
|
||||
app.eventStream.publish(Error(e, self, "error while processing " + message))
|
||||
system.eventStream.publish(Error(e, self, "error while processing " + message))
|
||||
//TODO FIXME How should problems here be handled?
|
||||
throw e
|
||||
}
|
||||
|
|
@ -284,7 +286,7 @@ private[akka] class ActorCell(
|
|||
case msg ⇒
|
||||
if (stopping) {
|
||||
// receiving Terminated in response to stopping children is too common to generate noise
|
||||
if (!msg.isInstanceOf[Terminated]) app.deadLetterMailbox.enqueue(self, messageHandle)
|
||||
if (!msg.isInstanceOf[Terminated]) system.deadLetterMailbox.enqueue(self, messageHandle)
|
||||
} else {
|
||||
actor(msg)
|
||||
}
|
||||
|
|
@ -292,7 +294,7 @@ private[akka] class ActorCell(
|
|||
currentMessage = null // reset current message after successful invocation
|
||||
} catch {
|
||||
case e ⇒
|
||||
app.eventStream.publish(Error(e, self, e.getMessage))
|
||||
system.eventStream.publish(Error(e, self, e.getMessage))
|
||||
|
||||
// prevent any further messages to be processed until the actor has been restarted
|
||||
dispatcher.suspend(this)
|
||||
|
|
@ -312,7 +314,7 @@ private[akka] class ActorCell(
|
|||
}
|
||||
} catch {
|
||||
case e ⇒
|
||||
app.eventStream.publish(Error(e, self, e.getMessage))
|
||||
system.eventStream.publish(Error(e, self, e.getMessage))
|
||||
throw e
|
||||
}
|
||||
}
|
||||
|
|
@ -330,11 +332,11 @@ private[akka] class ActorCell(
|
|||
}
|
||||
|
||||
def autoReceiveMessage(msg: Envelope) {
|
||||
if (app.AkkaConfig.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg))
|
||||
if (system.settings.DebugAutoReceive) system.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg))
|
||||
|
||||
if (stopping) msg.message match {
|
||||
case ChildTerminated ⇒ handleChildTerminated(sender)
|
||||
case _ ⇒ app.deadLetterMailbox.enqueue(self, msg)
|
||||
case _ ⇒ system.deadLetterMailbox.enqueue(self, msg)
|
||||
}
|
||||
else msg.message match {
|
||||
case HotSwap(code, discardOld) ⇒ become(code(self), discardOld)
|
||||
|
|
@ -347,8 +349,8 @@ private[akka] class ActorCell(
|
|||
}
|
||||
|
||||
private def doTerminate() {
|
||||
if (!app.provider.evict(self.path.toString))
|
||||
app.eventStream.publish(Warning(self, "evict of " + self.path.toString + " failed"))
|
||||
if (!system.provider.evict(self.path.toString))
|
||||
system.eventStream.publish(Warning(self, "evict of " + self.path.toString + " failed"))
|
||||
|
||||
dispatcher.detach(this)
|
||||
|
||||
|
|
@ -358,8 +360,8 @@ private[akka] class ActorCell(
|
|||
} finally {
|
||||
try {
|
||||
parent.tell(ChildTerminated, self)
|
||||
app.deathWatch.publish(Terminated(self))
|
||||
if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped"))
|
||||
system.deathWatch.publish(Terminated(self))
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped"))
|
||||
} finally {
|
||||
currentMessage = null
|
||||
clearActorFields()
|
||||
|
|
@ -369,7 +371,7 @@ private[akka] class ActorCell(
|
|||
|
||||
final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenStats.get(child) match {
|
||||
case Some(stats) ⇒ if (!props.faultHandler.handleFailure(child, cause, stats, childrenStats)) throw cause
|
||||
case None ⇒ app.eventStream.publish(Warning(self, "dropping Failed(" + cause + ") from unknown child"))
|
||||
case None ⇒ system.eventStream.publish(Warning(self, "dropping Failed(" + cause + ") from unknown child"))
|
||||
}
|
||||
|
||||
final def handleChildTerminated(child: ActorRef): Unit = {
|
||||
|
|
@ -387,7 +389,7 @@ private[akka] class ActorCell(
|
|||
val recvtimeout = receiveTimeout
|
||||
if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) {
|
||||
//Only reschedule if desired and there are currently no more messages to be processed
|
||||
futureTimeout = Some(app.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS))
|
||||
futureTimeout = Some(system.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,6 +4,8 @@
|
|||
|
||||
package akka.actor
|
||||
|
||||
import akka.remote.RemoteAddress
|
||||
|
||||
object ActorPath {
|
||||
final val separator = "/"
|
||||
|
||||
|
|
@ -12,14 +14,14 @@ object ActorPath {
|
|||
/**
|
||||
* Create an actor path from a string.
|
||||
*/
|
||||
def apply(app: ActorSystem, path: String): ActorPath =
|
||||
apply(app, split(path))
|
||||
def apply(system: ActorSystem, path: String): ActorPath =
|
||||
apply(system, split(path))
|
||||
|
||||
/**
|
||||
* Create an actor path from an iterable.
|
||||
*/
|
||||
def apply(app: ActorSystem, path: Iterable[String]): ActorPath =
|
||||
path.foldLeft(app.root)(_ / _)
|
||||
def apply(system: ActorSystem, path: Iterable[String]): ActorPath =
|
||||
path.foldLeft(system.rootPath)(_ / _)
|
||||
|
||||
/**
|
||||
* Split a string path into an iterable.
|
||||
|
|
@ -58,9 +60,9 @@ object ActorPath {
|
|||
*/
|
||||
trait ActorPath {
|
||||
/**
|
||||
* The akka application for this path.
|
||||
* The RemoteAddress for this path.
|
||||
*/
|
||||
def app: ActorSystem
|
||||
def remoteAddress: RemoteAddress
|
||||
|
||||
/**
|
||||
* The name of the actor that this path refers to.
|
||||
|
|
@ -78,9 +80,9 @@ trait ActorPath {
|
|||
def /(child: String): ActorPath
|
||||
|
||||
/**
|
||||
* Find the ActorRef for this path.
|
||||
* Recursively create a descendant’s path by appending all child names.
|
||||
*/
|
||||
def ref: Option[ActorRef]
|
||||
def /(child: Iterable[String]): ActorPath = (this /: child)(_ / _)
|
||||
|
||||
/**
|
||||
* String representation of this path. Different from toString for root path.
|
||||
|
|
@ -98,15 +100,13 @@ trait ActorPath {
|
|||
def isRoot: Boolean
|
||||
}
|
||||
|
||||
class RootActorPath(val app: ActorSystem) extends ActorPath {
|
||||
class RootActorPath(val remoteAddress: RemoteAddress) extends ActorPath {
|
||||
|
||||
def name: String = "/"
|
||||
|
||||
def parent: ActorPath = this
|
||||
|
||||
def /(child: String): ActorPath = new ChildActorPath(app, this, child)
|
||||
|
||||
def ref: Option[ActorRef] = app.actorFor(path)
|
||||
def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
|
||||
|
||||
def string: String = ""
|
||||
|
||||
|
|
@ -117,11 +117,9 @@ class RootActorPath(val app: ActorSystem) extends ActorPath {
|
|||
override def toString = ActorPath.separator
|
||||
}
|
||||
|
||||
class ChildActorPath(val app: ActorSystem, val parent: ActorPath, val name: String) extends ActorPath {
|
||||
class ChildActorPath(val remoteAddress: RemoteAddress, val parent: ActorPath, val name: String) extends ActorPath {
|
||||
|
||||
def /(child: String): ActorPath = new ChildActorPath(app, this, child)
|
||||
|
||||
def ref: Option[ActorRef] = app.actorFor(path)
|
||||
def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
|
||||
|
||||
def string: String = parent.string + ActorPath.separator + name
|
||||
|
||||
|
|
|
|||
|
|
@ -12,6 +12,8 @@ import akka.serialization.Serialization
|
|||
import java.net.InetSocketAddress
|
||||
import akka.remote.RemoteAddress
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.event.EventStream
|
||||
import akka.event.DeathWatch
|
||||
|
||||
/**
|
||||
* ActorRef is an immutable and serializable handle to an Actor.
|
||||
|
|
@ -160,7 +162,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class LocalActorRef private[akka] (
|
||||
app: ActorSystem,
|
||||
system: ActorSystemImpl,
|
||||
_props: Props,
|
||||
_supervisor: ActorRef,
|
||||
val path: ActorPath,
|
||||
|
|
@ -171,10 +173,19 @@ class LocalActorRef private[akka] (
|
|||
|
||||
def name = path.name
|
||||
|
||||
def address: String = app.address + path.toString
|
||||
def address: String = path.toString
|
||||
|
||||
/*
|
||||
* actorCell.start() publishes actorCell & this to the dispatcher, which
|
||||
* means that messages may be processed theoretically before the constructor
|
||||
* ends. The JMM guarantees visibility for final fields only after the end
|
||||
* of the constructor, so publish the actorCell safely by making it a
|
||||
* @volatile var which is NOT TO BE WRITTEN TO. The alternative would be to
|
||||
* move start() outside of the constructor, which would basically require
|
||||
* us to use purely factory methods for creating LocalActorRefs.
|
||||
*/
|
||||
@volatile
|
||||
private var actorCell = new ActorCell(app, this, _props, _supervisor, _receiveTimeout, _hotswap)
|
||||
private var actorCell = new ActorCell(system, this, _props, _supervisor, _receiveTimeout, _hotswap)
|
||||
actorCell.start()
|
||||
|
||||
/**
|
||||
|
|
@ -295,17 +306,17 @@ trait ScalaActorRef { ref: ActorRef ⇒
|
|||
*/
|
||||
|
||||
case class SerializedActorRef(hostname: String, port: Int, path: String) {
|
||||
import akka.serialization.Serialization.app
|
||||
import akka.serialization.Serialization.system
|
||||
|
||||
def this(remoteAddress: RemoteAddress, path: String) = this(remoteAddress.hostname, remoteAddress.port, path)
|
||||
def this(remoteAddress: InetSocketAddress, path: String) = this(remoteAddress.getAddress.getHostAddress, remoteAddress.getPort, path) //TODO FIXME REMOVE
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
def readResolve(): AnyRef = {
|
||||
if (app.value eq null) throw new IllegalStateException(
|
||||
if (system.value eq null) throw new IllegalStateException(
|
||||
"Trying to deserialize a serialized ActorRef without an ActorSystem in scope." +
|
||||
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }")
|
||||
app.value.provider.deserialize(this) match {
|
||||
" Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }")
|
||||
system.value.provider.deserialize(this) match {
|
||||
case Some(actor) ⇒ actor
|
||||
case None ⇒ throw new IllegalStateException("Could not deserialize ActorRef")
|
||||
}
|
||||
|
|
@ -344,31 +355,35 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
|
|||
object DeadLetterActorRef {
|
||||
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def readResolve(): AnyRef = Serialization.app.value.deadLetters
|
||||
private def readResolve(): AnyRef = Serialization.system.value.deadLetters
|
||||
}
|
||||
|
||||
val serialized = new SerializedDeadLetterActorRef
|
||||
}
|
||||
|
||||
class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
|
||||
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
|
||||
class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) extends MinimalActorRef {
|
||||
@volatile
|
||||
var brokenPromise: Future[Any] = _
|
||||
|
||||
private[akka] def init(dispatcher: MessageDispatcher) {
|
||||
brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher)
|
||||
}
|
||||
|
||||
override val name: String = "dead-letter"
|
||||
|
||||
// FIXME (actor path): put this under the sys guardian supervisor
|
||||
val path: ActorPath = app.root / "sys" / name
|
||||
|
||||
def address: String = app.address + path.toString
|
||||
def address: String = path.toString
|
||||
|
||||
override def isShutdown(): Boolean = true
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
|
||||
case d: DeadLetter ⇒ app.eventStream.publish(d)
|
||||
case _ ⇒ app.eventStream.publish(DeadLetter(message, sender, this))
|
||||
case d: DeadLetter ⇒ eventStream.publish(d)
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
|
||||
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
|
||||
app.eventStream.publish(DeadLetter(message, app.provider.dummyAskSender, this))
|
||||
eventStream.publish(DeadLetter(message, this, this))
|
||||
// leave this in: guard with good visibility against really stupid/weird errors
|
||||
assert(brokenPromise != null)
|
||||
brokenPromise
|
||||
}
|
||||
|
||||
|
|
@ -376,16 +391,15 @@ class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
|
|||
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
|
||||
}
|
||||
|
||||
abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef {
|
||||
abstract class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deathWatch: DeathWatch, timeout: Timeout, val dispatcher: MessageDispatcher) extends MinimalActorRef {
|
||||
final val result = new DefaultPromise[Any](timeout)(dispatcher)
|
||||
|
||||
// FIXME (actor path): put this under the tmp guardian supervisor
|
||||
val path: ActorPath = app.root / "tmp" / name
|
||||
override def name = path.name
|
||||
|
||||
def address: String = app.address + path.toString
|
||||
def address: String = path.toString
|
||||
|
||||
{
|
||||
val callback: Future[Any] ⇒ Unit = { _ ⇒ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
|
||||
val callback: Future[Any] ⇒ Unit = { _ ⇒ deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
|
||||
result onComplete callback
|
||||
result onTimeout callback
|
||||
}
|
||||
|
|
@ -411,5 +425,5 @@ abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = ap
|
|||
override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped"))
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def writeReplace(): AnyRef = app.provider.serialize(this)
|
||||
private def writeReplace(): AnyRef = provider.serialize(this)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,41 +5,59 @@
|
|||
package akka.actor
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import java.util.concurrent.{ TimeUnit, Executors }
|
||||
|
||||
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit }
|
||||
import scala.annotation.tailrec
|
||||
|
||||
import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer }
|
||||
|
||||
import akka.actor.Timeout.intToTimeout
|
||||
import akka.config.ConfigurationException
|
||||
import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise }
|
||||
import akka.event.{ Logging, DeathWatch, ActorClassification }
|
||||
import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise, Dispatcher, Mailbox, Envelope }
|
||||
import akka.event.{ Logging, DeathWatch, ActorClassification, EventStream }
|
||||
import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter }
|
||||
import akka.util.Helpers
|
||||
import akka.AkkaException
|
||||
import com.eaio.uuid.UUID
|
||||
|
||||
/**
|
||||
* Interface for all ActorRef providers to implement.
|
||||
*/
|
||||
trait ActorRefProvider {
|
||||
|
||||
def actorOf(props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(props, supervisor, name, false)
|
||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(system, props, supervisor, name, false)
|
||||
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef]
|
||||
|
||||
/**
|
||||
* What deployer will be used to resolve deployment configuration?
|
||||
*/
|
||||
def guardian: ActorRef
|
||||
|
||||
def systemGuardian: ActorRef
|
||||
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
// FIXME: remove/replace
|
||||
def nodename: String
|
||||
|
||||
def settings: ActorSystem.Settings
|
||||
|
||||
def init(system: ActorSystemImpl)
|
||||
|
||||
private[akka] def deployer: Deployer
|
||||
|
||||
private[akka] def scheduler: Scheduler
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
|
||||
/**
|
||||
* Create an Actor with the given name below the given supervisor.
|
||||
*/
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
|
||||
/**
|
||||
* Create an Actor with the given full path below the given supervisor.
|
||||
*
|
||||
* FIXME: Remove! this is dangerous!
|
||||
*/
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
|
||||
|
||||
/**
|
||||
* Remove this path from the lookup map.
|
||||
*/
|
||||
private[akka] def evict(path: String): Boolean
|
||||
|
||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef]
|
||||
|
|
@ -48,15 +66,16 @@ trait ActorRefProvider {
|
|||
|
||||
private[akka] def createDeathWatch(): DeathWatch
|
||||
|
||||
/**
|
||||
* Create AskActorRef to hook up message send to recipient with Future receiver.
|
||||
*/
|
||||
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any]
|
||||
|
||||
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef
|
||||
|
||||
/**
|
||||
* This Future is completed upon termination of this ActorRefProvider, which
|
||||
* is usually initiated by stopping the guardian via ActorSystem.stop().
|
||||
*/
|
||||
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
|
||||
|
||||
private[akka] def dummyAskSender: ActorRef
|
||||
|
||||
private[akka] def tempPath: String
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -64,9 +83,11 @@ trait ActorRefProvider {
|
|||
*/
|
||||
trait ActorRefFactory {
|
||||
|
||||
def provider: ActorRefProvider
|
||||
protected def systemImpl: ActorSystemImpl
|
||||
|
||||
def dispatcher: MessageDispatcher
|
||||
protected def provider: ActorRefProvider
|
||||
|
||||
protected def dispatcher: MessageDispatcher
|
||||
|
||||
/**
|
||||
* Father of all children created by this interface.
|
||||
|
|
@ -80,7 +101,7 @@ trait ActorRefFactory {
|
|||
Helpers.base64(l)
|
||||
}
|
||||
|
||||
def actorOf(props: Props): ActorRef = provider.actorOf(props, guardian, randomName, false)
|
||||
def actorOf(props: Props): ActorRef = provider.actorOf(systemImpl, props, guardian, randomName, false)
|
||||
|
||||
/*
|
||||
* TODO this will have to go at some point, because creating two actors with
|
||||
|
|
@ -90,7 +111,7 @@ trait ActorRefFactory {
|
|||
def actorOf(props: Props, name: String): ActorRef = {
|
||||
if (name == null || name == "" || name.startsWith("$"))
|
||||
throw new ActorInitializationException("actor name must not be null, empty or start with $")
|
||||
provider.actorOf(props, guardian, name, false)
|
||||
provider.actorOf(systemImpl, props, guardian, name, false)
|
||||
}
|
||||
|
||||
def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]))
|
||||
|
|
@ -104,6 +125,8 @@ trait ActorRefFactory {
|
|||
|
||||
def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() ⇒ creator.create()))
|
||||
|
||||
def actorFor(path: ActorPath): Option[ActorRef] = actorFor(path.path)
|
||||
|
||||
def actorFor(path: String): Option[ActorRef] = actorFor(ActorPath.split(path))
|
||||
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = provider.actorFor(path)
|
||||
|
|
@ -114,20 +137,37 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
|
|||
/**
|
||||
* Local ActorRef provider.
|
||||
*/
|
||||
class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
||||
class LocalActorRefProvider(
|
||||
val settings: ActorSystem.Settings,
|
||||
val rootPath: ActorPath,
|
||||
val eventStream: EventStream,
|
||||
val dispatcher: MessageDispatcher,
|
||||
val scheduler: Scheduler) extends ActorRefProvider {
|
||||
|
||||
val log = Logging(app.eventStream, this)
|
||||
val log = Logging(eventStream, this)
|
||||
|
||||
private[akka] val deployer: Deployer = new Deployer(app)
|
||||
|
||||
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher)
|
||||
|
||||
private[akka] val scheduler: Scheduler = { //TODO FIXME Make this configurable
|
||||
val s = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
|
||||
terminationFuture.onComplete(_ ⇒ s.stop())
|
||||
s
|
||||
// FIXME remove/replave (clustering shall not leak into akka-actor)
|
||||
val nodename: String = System.getProperty("akka.cluster.nodename") match {
|
||||
case null | "" ⇒ new UUID().toString
|
||||
case value ⇒ value
|
||||
}
|
||||
|
||||
private[akka] val deployer: Deployer = new Deployer(settings, eventStream, nodename)
|
||||
|
||||
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher)
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
private val tempNumber = new AtomicLong
|
||||
def tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement())
|
||||
private val tempNode = rootPath / "tmp"
|
||||
def tempPath = tempNode / tempName
|
||||
|
||||
// FIXME (actor path): this could become a cache for the new tree traversal actorFor
|
||||
// currently still used for tmp actors (e.g. ask actor refs)
|
||||
private val actors = new ConcurrentHashMap[String, AnyRef]
|
||||
|
||||
/**
|
||||
* Top-level anchor for the supervision hierarchy of this actor system. Will
|
||||
* receive only Supervise/ChildTerminated system messages or Failure message.
|
||||
|
|
@ -136,12 +176,12 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
@volatile
|
||||
var stopped = false
|
||||
|
||||
override val name = app.name + "-bubble-walker"
|
||||
override val name = "bubble-walker"
|
||||
|
||||
// FIXME (actor path): move the root path to the new root guardian
|
||||
val path = app.root
|
||||
val path = rootPath / name
|
||||
|
||||
val address = app.address + path.toString
|
||||
val address = path.toString
|
||||
|
||||
override def toString = name
|
||||
|
||||
|
|
@ -163,12 +203,57 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
}
|
||||
}
|
||||
|
||||
// FIXME (actor path): this could become a cache for the new tree traversal actorFor
|
||||
// currently still used for tmp actors (e.g. ask actor refs)
|
||||
private val actors = new ConcurrentHashMap[String, AnyRef]
|
||||
private class Guardian extends Actor {
|
||||
def receive = {
|
||||
case Terminated(_) ⇒ context.self.stop()
|
||||
}
|
||||
}
|
||||
private class SystemGuardian extends Actor {
|
||||
def receive = {
|
||||
case Terminated(_) ⇒
|
||||
eventStream.stopDefaultLoggers()
|
||||
context.self.stop()
|
||||
}
|
||||
}
|
||||
private val guardianFaultHandlingStrategy = {
|
||||
import akka.actor.FaultHandlingStrategy._
|
||||
OneForOneStrategy {
|
||||
case _: ActorKilledException ⇒ Stop
|
||||
case _: ActorInitializationException ⇒ Stop
|
||||
case _: Exception ⇒ Restart
|
||||
}
|
||||
}
|
||||
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
|
||||
|
||||
// FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "app" name for now)
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(app.guardian), path.tail)
|
||||
/*
|
||||
* The problem is that ActorRefs need a reference to the ActorSystem to
|
||||
* provide their service. Hence they cannot be created while the
|
||||
* constructors of ActorSystem and ActorRefProvider are still running.
|
||||
* The solution is to split out that last part into an init() method,
|
||||
* but it also requires these references to be @volatile.
|
||||
*/
|
||||
@volatile
|
||||
private var rootGuardian: ActorRef = _
|
||||
@volatile
|
||||
private var _guardian: ActorRef = _
|
||||
@volatile
|
||||
private var _systemGuardian: ActorRef = _
|
||||
def guardian = _guardian
|
||||
def systemGuardian = _systemGuardian
|
||||
|
||||
val deathWatch = createDeathWatch()
|
||||
|
||||
def init(system: ActorSystemImpl) {
|
||||
rootGuardian = actorOf(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true)
|
||||
_guardian = actorOf(system, guardianProps, rootGuardian, "app", true)
|
||||
_systemGuardian = actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
|
||||
// chain death watchers so that killing guardian stops the application
|
||||
deathWatch.subscribe(_systemGuardian, _guardian)
|
||||
deathWatch.subscribe(rootGuardian, _systemGuardian)
|
||||
}
|
||||
|
||||
// FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "system" name for now)
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(guardian), path.tail)
|
||||
|
||||
@tailrec
|
||||
private def findInTree(start: Option[ActorRef], path: Iterable[String]): Option[ActorRef] = {
|
||||
|
|
@ -193,12 +278,12 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
*/
|
||||
private[akka] def evict(path: String): Boolean = actors.remove(path) ne null
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(props, supervisor, supervisor.path / name, systemService)
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(system, props, supervisor, supervisor.path / name, systemService)
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
|
||||
val name = path.name
|
||||
val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout?
|
||||
val newFuture = Promise[ActorRef](5000)(dispatcher) // FIXME is this proper timeout?
|
||||
|
||||
actors.putIfAbsent(path.toString, newFuture) match {
|
||||
case null ⇒
|
||||
|
|
@ -207,7 +292,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
|
||||
// create a local actor
|
||||
case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, DeploymentConfig.LocalScope)) ⇒
|
||||
new LocalActorRef(app, props, supervisor, path, systemService) // create a local actor
|
||||
new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
|
||||
|
||||
// create a routed actor ref
|
||||
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.LocalScope)) ⇒
|
||||
|
|
@ -217,7 +302,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
case RouterType.Random ⇒ () ⇒ new RandomRouter
|
||||
case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter
|
||||
case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()(
|
||||
if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout)
|
||||
if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, settings.ActorTimeout)
|
||||
case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
|
||||
case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
|
||||
case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet")
|
||||
|
|
@ -226,10 +311,10 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
|
||||
val connections: Iterable[ActorRef] = (1 to nrOfInstances.factor) map { i ⇒
|
||||
val routedPath = path.parent / (path.name + ":" + i)
|
||||
new LocalActorRef(app, props, supervisor, routedPath, systemService)
|
||||
new LocalActorRef(system, props, supervisor, routedPath, systemService)
|
||||
}
|
||||
|
||||
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
|
||||
actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
|
||||
|
||||
case unknown ⇒ throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown)
|
||||
}
|
||||
|
|
@ -254,7 +339,7 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
/**
|
||||
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
|
||||
*/
|
||||
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
def actorOf(system: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
// FIXME: this needs to take supervision into account!
|
||||
|
||||
//FIXME clustering should be implemented by cluster actor ref provider
|
||||
|
|
@ -267,34 +352,26 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
// val localOnly = props.localOnly
|
||||
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
|
||||
// else new RoutedActorRef(props, address)
|
||||
new RoutedActorRef(app, props, supervisor, name)
|
||||
new RoutedActorRef(system, props, supervisor, name)
|
||||
}
|
||||
|
||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path))
|
||||
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(app.address, actor.path.toString)
|
||||
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(rootPath.remoteAddress, actor.path.toString)
|
||||
|
||||
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
|
||||
|
||||
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = {
|
||||
import akka.dispatch.DefaultPromise
|
||||
(if (within == null) app.AkkaConfig.ActorTimeout else within) match {
|
||||
(if (within == null) settings.ActorTimeout else within) match {
|
||||
case t if t.duration.length <= 0 ⇒
|
||||
new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout
|
||||
new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout
|
||||
case t ⇒
|
||||
val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this.path.toString) }
|
||||
val a = new AskActorRef(tempPath, this, deathWatch, t, dispatcher) { def whenDone() = actors.remove(this) }
|
||||
assert(actors.putIfAbsent(a.path.toString, a) eq null) //If this fails, we're in deep trouble
|
||||
recipient.tell(message, a)
|
||||
a.result
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] val dummyAskSender = new DeadLetterActorRef(app)
|
||||
|
||||
private val tempNumber = new AtomicLong
|
||||
def tempPath = {
|
||||
val l = tempNumber.getAndIncrement()
|
||||
"$_" + Helpers.base64(l)
|
||||
}
|
||||
}
|
||||
|
||||
class LocalDeathWatch extends DeathWatch with ActorClassification {
|
||||
|
|
|
|||
|
|
@ -6,19 +6,19 @@ package akka.actor
|
|||
import akka.config._
|
||||
import akka.actor._
|
||||
import akka.event._
|
||||
import akka.dispatch._
|
||||
import akka.util.duration._
|
||||
import java.net.InetAddress
|
||||
import com.eaio.uuid.UUID
|
||||
import akka.dispatch.{ Dispatchers, Future, Mailbox, Envelope, SystemMessage }
|
||||
import akka.util.Duration
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.serialization.Serialization
|
||||
import akka.remote.RemoteAddress
|
||||
import org.jboss.netty.akka.util.HashedWheelTimer
|
||||
import java.util.concurrent.{ Executors, TimeUnit }
|
||||
|
||||
object ActorSystem {
|
||||
|
||||
type AkkaConfig = a.AkkaConfig.type forSome { val a: ActorSystem }
|
||||
|
||||
val Version = "2.0-SNAPSHOT"
|
||||
|
||||
val envHome = System.getenv("AKKA_HOME") match {
|
||||
|
|
@ -61,26 +61,20 @@ object ActorSystem {
|
|||
|
||||
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
|
||||
|
||||
def apply(name: String, config: Configuration) = new ActorSystem(name, config)
|
||||
def create(name: String, config: Configuration): ActorSystem = apply(name, config)
|
||||
def apply(name: String, config: Configuration): ActorSystem = new ActorSystemImpl(name, config).start()
|
||||
|
||||
def apply(name: String): ActorSystem = new ActorSystem(name)
|
||||
def create(name: String): ActorSystem = apply(name)
|
||||
def apply(name: String): ActorSystem = apply(name, defaultConfig)
|
||||
|
||||
def apply(): ActorSystem = new ActorSystem()
|
||||
def create(): ActorSystem = apply()
|
||||
def apply(): ActorSystem = apply("default")
|
||||
|
||||
sealed trait ExitStatus
|
||||
case object Stopped extends ExitStatus
|
||||
case class Failed(cause: Throwable) extends ExitStatus
|
||||
|
||||
}
|
||||
|
||||
class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory {
|
||||
|
||||
def this(name: String) = this(name, ActorSystem.defaultConfig)
|
||||
def this() = this("default")
|
||||
|
||||
import ActorSystem._
|
||||
|
||||
object AkkaConfig {
|
||||
class Settings(val config: Configuration) {
|
||||
import config._
|
||||
val ConfigVersion = getString("akka.version", Version)
|
||||
|
||||
|
|
@ -126,90 +120,82 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
|
|||
|
||||
val FailureDetectorThreshold: Int = getInt("akka.remote.failure-detector.threshold", 8)
|
||||
val FailureDetectorMaxSampleSize: Int = getInt("akka.remote.failure-detector.max-sample-size", 1000)
|
||||
}
|
||||
|
||||
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true)
|
||||
|
||||
import AkkaConfig._
|
||||
|
||||
if (ConfigVersion != Version)
|
||||
throw new ConfigurationException("Akka JAR version [" + Version +
|
||||
"] does not match the provided config version [" + ConfigVersion + "]")
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
|
||||
import ActorSystem._
|
||||
|
||||
def name: String
|
||||
def settings: Settings
|
||||
def nodename: String
|
||||
|
||||
/**
|
||||
* Construct a path below the application guardian.
|
||||
*/
|
||||
def /(name: String): ActorPath
|
||||
def rootPath: ActorPath
|
||||
|
||||
val startTime = System.currentTimeMillis
|
||||
def uptime = (System.currentTimeMillis - startTime) / 1000
|
||||
|
||||
val nodename: String = System.getProperty("akka.cluster.nodename") match {
|
||||
case null | "" ⇒ new UUID().toString
|
||||
case value ⇒ value
|
||||
def eventStream: EventStream
|
||||
def log: LoggingAdapter
|
||||
|
||||
def deadLetters: ActorRef
|
||||
def deadLetterMailbox: Mailbox
|
||||
|
||||
// FIXME: Serialization should be an extension
|
||||
def serialization: Serialization
|
||||
// FIXME: TypedActor should be an extension
|
||||
def typedActor: TypedActor
|
||||
|
||||
def scheduler: Scheduler
|
||||
def dispatcherFactory: Dispatchers
|
||||
def dispatcher: MessageDispatcher
|
||||
|
||||
def registerOnTermination(code: ⇒ Unit)
|
||||
def registerOnTermination(code: Runnable)
|
||||
def stop()
|
||||
}
|
||||
|
||||
class ActorSystemImpl(val name: String, config: Configuration) extends ActorSystem {
|
||||
|
||||
import ActorSystem._
|
||||
|
||||
val settings = new Settings(config)
|
||||
|
||||
protected def systemImpl = this
|
||||
|
||||
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true)
|
||||
|
||||
import settings._
|
||||
|
||||
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
|
||||
case null | "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||
case value ⇒ value
|
||||
}, System.getProperty("akka.remote.port") match {
|
||||
case null | "" ⇒ AkkaConfig.RemoteServerPort
|
||||
case null | "" ⇒ settings.RemoteServerPort
|
||||
case value ⇒ value.toInt
|
||||
})
|
||||
|
||||
// this provides basic logging (to stdout) until .start() is called below
|
||||
val eventStream = new EventStream(DebugEventStream)
|
||||
eventStream.startStdoutLogger(AkkaConfig)
|
||||
val log = new BusLogging(eventStream, this)
|
||||
|
||||
// TODO correctly pull its config from the config
|
||||
val dispatcherFactory = new Dispatchers(this)
|
||||
|
||||
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
|
||||
|
||||
def scheduler = provider.scheduler
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val reflective = new ReflectiveAccess(this)
|
||||
eventStream.startStdoutLogger(settings)
|
||||
val log = new BusLogging(eventStream, this) // “this” used only for .getClass in tagging messages
|
||||
|
||||
/**
|
||||
* The root actor path for this application.
|
||||
*/
|
||||
val root: ActorPath = new RootActorPath(this)
|
||||
val rootPath: ActorPath = new RootActorPath(address)
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val provider: ActorRefProvider = reflective.createProvider
|
||||
|
||||
def terminationFuture: Future[ExitStatus] = provider.terminationFuture
|
||||
|
||||
private class Guardian extends Actor {
|
||||
def receive = {
|
||||
case Terminated(_) ⇒ context.self.stop()
|
||||
}
|
||||
}
|
||||
private class SystemGuardian extends Actor {
|
||||
def receive = {
|
||||
case Terminated(_) ⇒
|
||||
eventStream.stopDefaultLoggers()
|
||||
context.self.stop()
|
||||
}
|
||||
}
|
||||
private val guardianFaultHandlingStrategy = {
|
||||
import akka.actor.FaultHandlingStrategy._
|
||||
OneForOneStrategy {
|
||||
case _: ActorKilledException ⇒ Stop
|
||||
case _: ActorInitializationException ⇒ Stop
|
||||
case _: Exception ⇒ Restart
|
||||
}
|
||||
}
|
||||
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
|
||||
|
||||
private val rootGuardian: ActorRef =
|
||||
provider.actorOf(guardianProps, provider.theOneWhoWalksTheBubblesOfSpaceTime, root, true)
|
||||
|
||||
protected[akka] val guardian: ActorRef =
|
||||
provider.actorOf(guardianProps, rootGuardian, "app", true)
|
||||
|
||||
protected[akka] val systemGuardian: ActorRef =
|
||||
provider.actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val deadLetters = new DeadLetterActorRef(this)
|
||||
val deadLetters = new DeadLetterActorRef(eventStream, rootPath / "nul")
|
||||
val deadLetterMailbox = new Mailbox(null) {
|
||||
becomeClosed()
|
||||
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
|
||||
|
|
@ -221,34 +207,70 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
|
|||
override def numberOfMessages = 0
|
||||
}
|
||||
|
||||
val deathWatch = provider.createDeathWatch()
|
||||
// FIXME make this configurable
|
||||
val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
|
||||
|
||||
// chain death watchers so that killing guardian stops the application
|
||||
deathWatch.subscribe(systemGuardian, guardian)
|
||||
deathWatch.subscribe(rootGuardian, systemGuardian)
|
||||
// TODO correctly pull its config from the config
|
||||
val dispatcherFactory = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler))
|
||||
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
|
||||
|
||||
deadLetters.init(dispatcher)
|
||||
|
||||
val provider: ActorRefProvider = {
|
||||
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(b) ⇒ b
|
||||
}
|
||||
val arguments = List(
|
||||
classOf[Settings] -> settings,
|
||||
classOf[ActorPath] -> rootPath,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[MessageDispatcher] -> dispatcher,
|
||||
classOf[Scheduler] -> scheduler)
|
||||
val types: Array[Class[_]] = arguments map (_._1) toArray
|
||||
val values: Array[AnyRef] = arguments map (_._2) toArray
|
||||
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
}
|
||||
|
||||
def terminationFuture: Future[ExitStatus] = provider.terminationFuture
|
||||
def guardian: ActorRef = provider.guardian
|
||||
def systemGuardian: ActorRef = provider.systemGuardian
|
||||
def deathWatch: DeathWatch = provider.deathWatch
|
||||
def nodename: String = provider.nodename
|
||||
|
||||
terminationFuture.onComplete(_ ⇒ scheduler.stop())
|
||||
terminationFuture.onComplete(_ ⇒ dispatcher.shutdown())
|
||||
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
@volatile
|
||||
private var _typedActor: TypedActor = _
|
||||
def typedActor = _typedActor
|
||||
|
||||
def /(actorName: String): ActorPath = guardian.path / actorName
|
||||
|
||||
def start(): this.type = {
|
||||
if (_serialization != null) throw new IllegalStateException("cannot initialize ActorSystemImpl twice!")
|
||||
_serialization = new Serialization(this)
|
||||
_typedActor = new TypedActor(settings, _serialization)
|
||||
provider.init(this)
|
||||
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
|
||||
eventStream.start(this)
|
||||
eventStream.startDefaultLoggers(this, AkkaConfig)
|
||||
eventStream.startDefaultLoggers(this)
|
||||
this
|
||||
}
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor
|
||||
val deployer = new Deployer(this)
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val typedActor = new TypedActor(this)
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val serialization = new Serialization(this)
|
||||
|
||||
/**
|
||||
* Create an actor path under the application supervisor (/app).
|
||||
*/
|
||||
def /(actorName: String): ActorPath = guardian.path / actorName
|
||||
def registerOnTermination(code: ⇒ Unit) { terminationFuture onComplete (_ ⇒ code) }
|
||||
def registerOnTermination(code: Runnable) { terminationFuture onComplete (_ ⇒ code.run) }
|
||||
|
||||
// TODO shutdown all that other stuff, whatever that may be
|
||||
def stop() {
|
||||
guardian.stop()
|
||||
}
|
||||
|
||||
terminationFuture.onComplete(_ ⇒ dispatcher.shutdown())
|
||||
}
|
||||
|
|
|
|||
|
|
@ -14,14 +14,14 @@ import akka.util.Bootable
|
|||
*/
|
||||
trait BootableActorLoaderService extends Bootable {
|
||||
|
||||
def app: ActorSystem
|
||||
def system: ActorSystem
|
||||
|
||||
val BOOT_CLASSES = app.AkkaConfig.BootClasses
|
||||
val BOOT_CLASSES = system.settings.BootClasses
|
||||
lazy val applicationLoader = createApplicationClassLoader()
|
||||
|
||||
protected def createApplicationClassLoader(): Option[ClassLoader] = Some({
|
||||
if (app.AkkaConfig.Home.isDefined) {
|
||||
val DEPLOY = app.AkkaConfig.Home.get + "/deploy"
|
||||
if (system.settings.Home.isDefined) {
|
||||
val DEPLOY = system.settings.Home.get + "/deploy"
|
||||
val DEPLOY_DIR = new File(DEPLOY)
|
||||
if (!DEPLOY_DIR.exists) {
|
||||
System.exit(-1)
|
||||
|
|
@ -59,11 +59,11 @@ trait BootableActorLoaderService extends Bootable {
|
|||
super.onUnload()
|
||||
|
||||
// FIXME shutdown all actors
|
||||
// app.registry.local.shutdownAll
|
||||
// system.registry.local.shutdownAll
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API for the default JAX-RS/Mist Initializer
|
||||
*/
|
||||
class DefaultBootableActorLoaderService(val app: ActorSystem) extends BootableActorLoaderService
|
||||
class DefaultBootableActorLoaderService(val system: ActorSystem) extends BootableActorLoaderService
|
||||
|
|
|
|||
|
|
@ -5,9 +5,7 @@
|
|||
package akka.actor
|
||||
|
||||
import collection.immutable.Seq
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import akka.event.Logging
|
||||
import akka.actor.DeploymentConfig._
|
||||
import akka.AkkaException
|
||||
|
|
@ -15,6 +13,7 @@ import akka.config.{ Configuration, ConfigurationException }
|
|||
import akka.util.Duration
|
||||
import java.net.InetSocketAddress
|
||||
import akka.remote.RemoteAddress
|
||||
import akka.event.EventStream
|
||||
|
||||
trait ActorDeployer {
|
||||
private[akka] def init(deployments: Seq[Deploy]): Unit
|
||||
|
|
@ -34,10 +33,10 @@ trait ActorDeployer {
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Deployer(val app: ActorSystem) extends ActorDeployer {
|
||||
class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream, val nodename: String) extends ActorDeployer {
|
||||
|
||||
val deploymentConfig = new DeploymentConfig(app)
|
||||
val log = Logging(app.eventStream, this)
|
||||
val deploymentConfig = new DeploymentConfig(nodename)
|
||||
val log = Logging(eventStream, this)
|
||||
|
||||
val instance: ActorDeployer = {
|
||||
val deployer = new LocalDeployer()
|
||||
|
|
@ -86,7 +85,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer {
|
|||
|
||||
private[akka] def pathsInConfig: List[String] = {
|
||||
val deploymentPath = "akka.actor.deployment"
|
||||
app.config.getSection(deploymentPath) match {
|
||||
settings.config.getSection(deploymentPath) match {
|
||||
case None ⇒ Nil
|
||||
case Some(pathConfig) ⇒
|
||||
pathConfig.map.keySet
|
||||
|
|
@ -98,7 +97,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer {
|
|||
/**
|
||||
* Lookup deployment in 'akka.conf' configuration file.
|
||||
*/
|
||||
private[akka] def lookupInConfig(path: String, configuration: Configuration = app.config): Option[Deploy] = {
|
||||
private[akka] def lookupInConfig(path: String, configuration: Configuration = settings.config): Option[Deploy] = {
|
||||
import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor }
|
||||
|
||||
// --------------------------------
|
||||
|
|
|
|||
|
|
@ -217,13 +217,13 @@ object DeploymentConfig {
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class DeploymentConfig(val app: ActorSystem) {
|
||||
class DeploymentConfig(val nodename: String) {
|
||||
|
||||
import DeploymentConfig._
|
||||
|
||||
case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(app.nodename)), replication: ReplicationScheme = Transient) extends Scope
|
||||
case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(nodename)), replication: ReplicationScheme = Transient) extends Scope
|
||||
|
||||
def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == app.nodename)
|
||||
def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == nodename)
|
||||
|
||||
def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match {
|
||||
case Deploy(_, _, _, _, ClusterScope(_, replicationScheme)) ⇒ Some(replicationScheme)
|
||||
|
|
|
|||
|
|
@ -28,14 +28,14 @@ object FSM {
|
|||
case object StateTimeout
|
||||
case class TimeoutMarker(generation: Long)
|
||||
|
||||
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: ActorSystem) {
|
||||
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) {
|
||||
private var ref: Option[Cancellable] = _
|
||||
|
||||
def schedule(actor: ActorRef, timeout: Duration) {
|
||||
if (repeat) {
|
||||
ref = Some(app.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit))
|
||||
ref = Some(system.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit))
|
||||
} else {
|
||||
ref = Some(app.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit))
|
||||
ref = Some(system.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -188,7 +188,7 @@ trait FSM[S, D] extends ListenerManagement {
|
|||
type Timeout = Option[Duration]
|
||||
type TransitionHandler = PartialFunction[(S, S), Unit]
|
||||
|
||||
val log = Logging(app.eventStream, context.self)
|
||||
val log = Logging(system, context.self)
|
||||
|
||||
/**
|
||||
* ****************************************
|
||||
|
|
@ -522,7 +522,7 @@ trait FSM[S, D] extends ListenerManagement {
|
|||
if (timeout.isDefined) {
|
||||
val t = timeout.get
|
||||
if (t.finite_? && t.length >= 0) {
|
||||
timeoutFuture = Some(app.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
|
||||
timeoutFuture = Some(system.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -565,7 +565,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
|
|||
|
||||
def logDepth: Int = 0
|
||||
|
||||
private val debugEvent = context.app.AkkaConfig.FsmDebugEvent
|
||||
private val debugEvent = system.settings.FsmDebugEvent
|
||||
|
||||
private val events = new Array[Event](logDepth)
|
||||
private val states = new Array[AnyRef](logDepth)
|
||||
|
|
|
|||
|
|
@ -253,7 +253,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor {
|
|||
var worker: IOWorker = _
|
||||
|
||||
override def preStart {
|
||||
worker = new IOWorker(app, self, bufferSize)
|
||||
worker = new IOWorker(system, self, bufferSize)
|
||||
worker.start()
|
||||
}
|
||||
|
||||
|
|
@ -290,7 +290,7 @@ private[akka] object IOWorker {
|
|||
case object Shutdown extends Request
|
||||
}
|
||||
|
||||
private[akka] class IOWorker(app: ActorSystem, ioManager: ActorRef, val bufferSize: Int) {
|
||||
private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val bufferSize: Int) {
|
||||
import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT }
|
||||
import IOWorker._
|
||||
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ object TypedActor {
|
|||
* This class represents a Method call, and has a reference to the Method to be called and the parameters to supply
|
||||
* It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized
|
||||
*/
|
||||
case class MethodCall(app: ActorSystem, method: Method, parameters: Array[AnyRef]) {
|
||||
case class MethodCall(ser: Serialization, method: Method, parameters: Array[AnyRef]) {
|
||||
|
||||
def isOneWay = method.getReturnType == java.lang.Void.TYPE
|
||||
def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType)
|
||||
|
|
@ -40,7 +40,7 @@ object TypedActor {
|
|||
case null ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, null, null)
|
||||
case ps if ps.length == 0 ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array[Serializer.Identifier](), Array[Array[Byte]]())
|
||||
case ps ⇒
|
||||
val serializers: Array[Serializer] = ps map app.serialization.findSerializerFor
|
||||
val serializers: Array[Serializer] = ps map ser.findSerializerFor
|
||||
val serializedParameters: Array[Array[Byte]] = Array.ofDim[Array[Byte]](serializers.length)
|
||||
for (i ← 0 until serializers.length)
|
||||
serializedParameters(i) = serializers(i) toBinary parameters(i) //Mutable for the sake of sanity
|
||||
|
|
@ -57,17 +57,17 @@ object TypedActor {
|
|||
//TODO implement writeObject and readObject to serialize
|
||||
//TODO Possible optimization is to special encode the parameter-types to conserve space
|
||||
private def readResolve(): AnyRef = {
|
||||
val app = akka.serialization.Serialization.app.value
|
||||
if (app eq null) throw new IllegalStateException(
|
||||
val system = akka.serialization.Serialization.system.value
|
||||
if (system eq null) throw new IllegalStateException(
|
||||
"Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." +
|
||||
" Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }")
|
||||
MethodCall(app, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match {
|
||||
" Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }")
|
||||
MethodCall(system.serialization, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match {
|
||||
case null ⇒ null
|
||||
case a if a.length == 0 ⇒ Array[AnyRef]()
|
||||
case a ⇒
|
||||
val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity
|
||||
for (i ← 0 until a.length) {
|
||||
deserializedParameters(i) = app.serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i))
|
||||
deserializedParameters(i) = system.serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i))
|
||||
}
|
||||
deserializedParameters
|
||||
})
|
||||
|
|
@ -101,22 +101,22 @@ object TypedActor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the akka app (for a TypedActor) when inside a method call in a TypedActor.
|
||||
* Returns the akka system (for a TypedActor) when inside a method call in a TypedActor.
|
||||
*/
|
||||
def app = appReference.get match {
|
||||
case null ⇒ throw new IllegalStateException("Calling TypedActor.app outside of a TypedActor implementation method!")
|
||||
def system = appReference.get match {
|
||||
case null ⇒ throw new IllegalStateException("Calling TypedActor.system outside of a TypedActor implementation method!")
|
||||
case some ⇒ some
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the default dispatcher (for a TypedActor) when inside a method call in a TypedActor.
|
||||
*/
|
||||
implicit def dispatcher = app.dispatcher
|
||||
implicit def dispatcher = system.dispatcher
|
||||
|
||||
/**
|
||||
* Returns the default timeout (for a TypedActor) when inside a method call in a TypedActor.
|
||||
*/
|
||||
implicit def timeout = app.AkkaConfig.ActorTimeout
|
||||
implicit def timeout = system.settings.ActorTimeout
|
||||
}
|
||||
|
||||
trait TypedActorFactory { this: ActorRefFactory ⇒
|
||||
|
|
@ -264,7 +264,7 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
|
|||
*
|
||||
* TypedActors needs, just like Actors, to be Stopped when they are no longer needed, use TypedActor.stop(proxy)
|
||||
*/
|
||||
class TypedActor(val app: ActorSystem) {
|
||||
class TypedActor(val settings: ActorSystem.Settings, var ser: Serialization) {
|
||||
|
||||
import TypedActor.MethodCall
|
||||
/**
|
||||
|
|
@ -313,7 +313,7 @@ class TypedActor(val app: ActorSystem) {
|
|||
//Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
|
||||
val actorVar = new AtomVar[ActorRef](null)
|
||||
val timeout = props.timeout match {
|
||||
case Props.`defaultTimeout` ⇒ app.AkkaConfig.ActorTimeout
|
||||
case Props.`defaultTimeout` ⇒ settings.ActorTimeout
|
||||
case x ⇒ x
|
||||
}
|
||||
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T]
|
||||
|
|
@ -330,7 +330,7 @@ class TypedActor(val app: ActorSystem) {
|
|||
def receive = {
|
||||
case m: MethodCall ⇒
|
||||
TypedActor.selfReference set proxyVar.get
|
||||
TypedActor.appReference set app
|
||||
TypedActor.appReference set system
|
||||
try {
|
||||
if (m.isOneWay) m(me)
|
||||
else {
|
||||
|
|
@ -365,7 +365,7 @@ class TypedActor(val app: ActorSystem) {
|
|||
case "equals" ⇒ (args.length == 1 && (proxy eq args(0)) || actor == getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean
|
||||
case "hashCode" ⇒ actor.hashCode.asInstanceOf[AnyRef]
|
||||
case _ ⇒
|
||||
MethodCall(app, method, args) match {
|
||||
MethodCall(ser, method, args) match {
|
||||
case m if m.isOneWay ⇒ actor ! m; null //Null return value
|
||||
case m if m.returnsFuture_? ⇒ actor.?(m, timeout)
|
||||
case m if m.returnsJOption_? || m.returnsOption_? ⇒
|
||||
|
|
|
|||
|
|
@ -8,10 +8,6 @@ package object actor {
|
|||
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
|
||||
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
|
||||
|
||||
// actor path can be used as an actor ref (note: does a lookup in the app using path.ref)
|
||||
implicit def actorPath2Ref(path: ActorPath): ActorRef = path.ref.getOrElse(path.app.deadLetters)
|
||||
implicit def actorPath2ScalaRef(path: ActorPath): ScalaActorRef = actorPath2Ref(path).asInstanceOf[ScalaActorRef]
|
||||
|
||||
type Uuid = com.eaio.uuid.UUID
|
||||
|
||||
def newUuid(): Uuid = new Uuid()
|
||||
|
|
|
|||
|
|
@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) {
|
|||
*/
|
||||
object NodeAddress {
|
||||
def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName)
|
||||
def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename)
|
||||
def apply(system: ActorSystem): NodeAddress = new NodeAddress(system.settings.ClusterName, system.nodename)
|
||||
|
||||
def unapply(other: Any) = other match {
|
||||
case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName))
|
||||
|
|
|
|||
|
|
@ -54,7 +54,7 @@ class FilesystemImporter(val baseDir: String) extends Importer {
|
|||
|
||||
/**
|
||||
* An Importer that looks for imported config files in the java resources
|
||||
* of the system class loader (usually the jar used to launch this app).
|
||||
* of the system class loader (usually the jar used to launch this system).
|
||||
*/
|
||||
class ResourceImporter(classLoader: ClassLoader) extends Importer {
|
||||
def importFile(filename: String): String = {
|
||||
|
|
|
|||
|
|
@ -14,6 +14,8 @@ import akka.actor._
|
|||
import akka.actor.ActorSystem
|
||||
import locks.ReentrantLock
|
||||
import scala.annotation.tailrec
|
||||
import akka.event.EventStream
|
||||
import akka.actor.ActorSystem.Settings
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
|
|
@ -62,12 +64,12 @@ case class Supervise(child: ActorRef) extends SystemMessage // sent to superviso
|
|||
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsWatching
|
||||
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsWatching
|
||||
|
||||
final case class TaskInvocation(app: ActorSystem, function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable {
|
||||
final case class TaskInvocation(eventStream: EventStream, function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable {
|
||||
def run() {
|
||||
try {
|
||||
function()
|
||||
} catch {
|
||||
case e ⇒ app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
case e ⇒ eventStream.publish(Error(e, this, e.getMessage))
|
||||
} finally {
|
||||
cleanup()
|
||||
}
|
||||
|
|
@ -79,26 +81,23 @@ object MessageDispatcher {
|
|||
val SCHEDULED = 1
|
||||
val RESCHEDULED = 2
|
||||
|
||||
implicit def defaultDispatcher(implicit app: ActorSystem) = app.dispatcher
|
||||
implicit def defaultDispatcher(implicit system: ActorSystem) = system.dispatcher
|
||||
}
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDispatcher with Serializable {
|
||||
abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Serializable {
|
||||
|
||||
import MessageDispatcher._
|
||||
import AbstractMessageDispatcher.{ inhabitantsUpdater, shutdownScheduleUpdater }
|
||||
import prerequisites._
|
||||
|
||||
/**
|
||||
* Creates and returns a mailbox for the given actor.
|
||||
*/
|
||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox
|
||||
|
||||
/**
|
||||
* a blackhole mailbox for the purpose of replacing the real one upon actor termination
|
||||
*/
|
||||
import app.deadLetterMailbox
|
||||
|
||||
/**
|
||||
* Name of this dispatcher.
|
||||
*/
|
||||
|
|
@ -119,7 +118,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
|
|||
}
|
||||
|
||||
protected[akka] final def dispatchTask(block: () ⇒ Unit) {
|
||||
val invocation = TaskInvocation(app, block, taskCleanup)
|
||||
val invocation = TaskInvocation(eventStream, block, taskCleanup)
|
||||
inhabitantsUpdater.incrementAndGet(this)
|
||||
try {
|
||||
executeTask(invocation)
|
||||
|
|
@ -136,7 +135,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
|
|||
shutdownScheduleUpdater.get(this) match {
|
||||
case UNSCHEDULED ⇒
|
||||
if (shutdownScheduleUpdater.compareAndSet(this, UNSCHEDULED, SCHEDULED)) {
|
||||
app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
|
||||
scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
|
||||
()
|
||||
} else ifSensibleToDoSoThenScheduleShutdown()
|
||||
case SCHEDULED ⇒
|
||||
|
|
@ -211,7 +210,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
|
|||
}
|
||||
case RESCHEDULED ⇒
|
||||
if (shutdownScheduleUpdater.compareAndSet(MessageDispatcher.this, RESCHEDULED, SCHEDULED))
|
||||
app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
|
||||
scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
|
||||
else run()
|
||||
}
|
||||
}
|
||||
|
|
@ -289,29 +288,31 @@ abstract class MessageDispatcher(val app: ActorSystem) extends AbstractMessageDi
|
|||
/**
|
||||
* Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig
|
||||
*/
|
||||
abstract class MessageDispatcherConfigurator(val app: ActorSystem) {
|
||||
abstract class MessageDispatcherConfigurator() {
|
||||
/**
|
||||
* Returns an instance of MessageDispatcher given a Configuration
|
||||
*/
|
||||
def configure(config: Configuration): MessageDispatcher
|
||||
def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher
|
||||
|
||||
def mailboxType(config: Configuration): MailboxType = {
|
||||
val capacity = config.getInt("mailbox-capacity", app.AkkaConfig.MailboxCapacity)
|
||||
def mailboxType(config: Configuration, settings: Settings): MailboxType = {
|
||||
val capacity = config.getInt("mailbox-capacity", settings.MailboxCapacity)
|
||||
if (capacity < 1) UnboundedMailbox()
|
||||
else {
|
||||
val duration = Duration(
|
||||
config.getInt("mailbox-push-timeout-time", app.AkkaConfig.MailboxPushTimeout.toMillis.toInt),
|
||||
app.AkkaConfig.DefaultTimeUnit)
|
||||
config.getInt("mailbox-push-timeout-time", settings.MailboxPushTimeout.toMillis.toInt),
|
||||
settings.DefaultTimeUnit)
|
||||
BoundedMailbox(capacity, duration)
|
||||
}
|
||||
}
|
||||
|
||||
def configureThreadPool(config: Configuration, createDispatcher: ⇒ (ThreadPoolConfig) ⇒ MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
|
||||
def configureThreadPool(config: Configuration,
|
||||
settings: Settings,
|
||||
createDispatcher: ⇒ (ThreadPoolConfig) ⇒ MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
|
||||
import ThreadPoolConfigDispatcherBuilder.conf_?
|
||||
|
||||
//Apply the following options to the config if they are present in the config
|
||||
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(app)).configure(
|
||||
conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, app.AkkaConfig.DefaultTimeUnit))),
|
||||
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()).configure(
|
||||
conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, settings.DefaultTimeUnit))),
|
||||
conf_?(config getDouble "core-pool-size-factor")(factor ⇒ _.setCorePoolSizeFromFactor(factor)),
|
||||
conf_?(config getDouble "max-pool-size-factor")(factor ⇒ _.setMaxPoolSizeFromFactor(factor)),
|
||||
conf_?(config getBool "allow-core-timeout")(allow ⇒ _.setAllowCoreThreadTimeout(allow)),
|
||||
|
|
|
|||
|
|
@ -10,6 +10,8 @@ import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, Concur
|
|||
import java.util.{ Comparator, Queue }
|
||||
import annotation.tailrec
|
||||
import akka.actor.ActorSystem
|
||||
import akka.event.EventStream
|
||||
import akka.actor.Scheduler
|
||||
|
||||
/**
|
||||
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
|
||||
|
|
@ -28,16 +30,14 @@ import akka.actor.ActorSystem
|
|||
* @author Viktor Klang
|
||||
*/
|
||||
class BalancingDispatcher(
|
||||
_app: ActorSystem,
|
||||
_prerequisites: DispatcherPrerequisites,
|
||||
_name: String,
|
||||
throughput: Int,
|
||||
throughputDeadlineTime: Int,
|
||||
mailboxType: MailboxType,
|
||||
config: ThreadPoolConfig,
|
||||
_timeoutMs: Long)
|
||||
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
|
||||
|
||||
import app.deadLetterMailbox
|
||||
extends Dispatcher(_prerequisites, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
|
||||
|
||||
private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
|
||||
|
||||
|
|
@ -82,7 +82,7 @@ class BalancingDispatcher(
|
|||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
deadLetterMailbox.systemEnqueue(actor.self, message)
|
||||
prerequisites.deadLetterMailbox.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,6 +9,8 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue }
|
||||
import akka.actor.{ ActorCell, ActorKilledException }
|
||||
import akka.actor.ActorSystem
|
||||
import akka.event.EventStream
|
||||
import akka.actor.Scheduler
|
||||
|
||||
/**
|
||||
* Default settings are:
|
||||
|
|
@ -62,14 +64,14 @@ import akka.actor.ActorSystem
|
|||
* Larger values (or zero or negative) increase throughput, smaller values increase fairness
|
||||
*/
|
||||
class Dispatcher(
|
||||
_app: ActorSystem,
|
||||
_prerequisites: DispatcherPrerequisites,
|
||||
val name: String,
|
||||
val throughput: Int,
|
||||
val throughputDeadlineTime: Int,
|
||||
val mailboxType: MailboxType,
|
||||
executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
|
||||
val timeoutMs: Long)
|
||||
extends MessageDispatcher(_app) {
|
||||
extends MessageDispatcher(_prerequisites) {
|
||||
|
||||
protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name)
|
||||
protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate {
|
||||
|
|
@ -97,7 +99,7 @@ class Dispatcher(
|
|||
executorService.get() execute invocation
|
||||
} catch {
|
||||
case e2: RejectedExecutionException ⇒
|
||||
app.eventStream.publish(Warning(this, e2.toString))
|
||||
prerequisites.eventStream.publish(Warning(this, e2.toString))
|
||||
throw e2
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -10,6 +10,20 @@ import akka.util.{ Duration, ReflectiveAccess }
|
|||
import akka.config.Configuration
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.actor.ActorSystem
|
||||
import akka.event.EventStream
|
||||
import akka.actor.Scheduler
|
||||
import akka.actor.ActorSystem.Settings
|
||||
|
||||
trait DispatcherPrerequisites {
|
||||
def eventStream: EventStream
|
||||
def deadLetterMailbox: Mailbox
|
||||
def scheduler: Scheduler
|
||||
}
|
||||
|
||||
case class DefaultDispatcherPrerequisites(
|
||||
val eventStream: EventStream,
|
||||
val deadLetterMailbox: Mailbox,
|
||||
val scheduler: Scheduler) extends DispatcherPrerequisites
|
||||
|
||||
/**
|
||||
* Scala API. Dispatcher factory.
|
||||
|
|
@ -41,15 +55,16 @@ import akka.actor.ActorSystem
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Dispatchers(val app: ActorSystem) {
|
||||
val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
|
||||
class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: DispatcherPrerequisites) {
|
||||
|
||||
val ThroughputDeadlineTimeMillis = settings.DispatcherThroughputDeadlineTime.toMillis.toInt
|
||||
val MailboxType: MailboxType =
|
||||
if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
|
||||
else BoundedMailbox(app.AkkaConfig.MailboxCapacity, app.AkkaConfig.MailboxPushTimeout)
|
||||
val DispatcherShutdownMillis = app.AkkaConfig.DispatcherDefaultShutdown.toMillis
|
||||
if (settings.MailboxCapacity < 1) UnboundedMailbox()
|
||||
else BoundedMailbox(settings.MailboxCapacity, settings.MailboxPushTimeout)
|
||||
val DispatcherShutdownMillis = settings.DispatcherDefaultShutdown.toMillis
|
||||
|
||||
lazy val defaultGlobalDispatcher =
|
||||
app.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
|
||||
settings.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
|
||||
|
||||
/**
|
||||
* Creates an thread based dispatcher serving a single actor through the same single thread.
|
||||
|
|
@ -58,8 +73,8 @@ class Dispatchers(val app: ActorSystem) {
|
|||
* E.g. each actor consumes its own thread.
|
||||
*/
|
||||
def newPinnedDispatcher(actor: LocalActorRef) = actor match {
|
||||
case null ⇒ new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis)
|
||||
case some ⇒ new PinnedDispatcher(app, some.underlying, some.address, MailboxType, DispatcherShutdownMillis)
|
||||
case null ⇒ new PinnedDispatcher(prerequisites, null, "anon", MailboxType, DispatcherShutdownMillis)
|
||||
case some ⇒ new PinnedDispatcher(prerequisites, some.underlying, some.address, MailboxType, DispatcherShutdownMillis)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -69,8 +84,8 @@ class Dispatchers(val app: ActorSystem) {
|
|||
* E.g. each actor consumes its own thread.
|
||||
*/
|
||||
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
|
||||
case null ⇒ new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis)
|
||||
case some ⇒ new PinnedDispatcher(app, some.underlying, some.address, mailboxType, DispatcherShutdownMillis)
|
||||
case null ⇒ new PinnedDispatcher(prerequisites, null, "anon", mailboxType, DispatcherShutdownMillis)
|
||||
case some ⇒ new PinnedDispatcher(prerequisites, some.underlying, some.address, mailboxType, DispatcherShutdownMillis)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -79,7 +94,7 @@ class Dispatchers(val app: ActorSystem) {
|
|||
* E.g. each actor consumes its own thread.
|
||||
*/
|
||||
def newPinnedDispatcher(name: String, mailboxType: MailboxType) =
|
||||
new PinnedDispatcher(app, null, name, mailboxType, DispatcherShutdownMillis)
|
||||
new PinnedDispatcher(prerequisites, null, name, mailboxType, DispatcherShutdownMillis)
|
||||
|
||||
/**
|
||||
* Creates an thread based dispatcher serving a single actor through the same single thread.
|
||||
|
|
@ -87,7 +102,7 @@ class Dispatchers(val app: ActorSystem) {
|
|||
* E.g. each actor consumes its own thread.
|
||||
*/
|
||||
def newPinnedDispatcher(name: String) =
|
||||
new PinnedDispatcher(app, null, name, MailboxType, DispatcherShutdownMillis)
|
||||
new PinnedDispatcher(prerequisites, null, name, MailboxType, DispatcherShutdownMillis)
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -95,8 +110,8 @@ class Dispatchers(val app: ActorSystem) {
|
|||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newDispatcher(name: String) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(app, name, app.AkkaConfig.DispatcherThroughput,
|
||||
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(prerequisites, name, settings.DispatcherThroughput,
|
||||
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -105,7 +120,7 @@ class Dispatchers(val app: ActorSystem) {
|
|||
*/
|
||||
def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
new Dispatcher(prerequisites, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -114,7 +129,7 @@ class Dispatchers(val app: ActorSystem) {
|
|||
*/
|
||||
def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
new Dispatcher(prerequisites, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -122,8 +137,8 @@ class Dispatchers(val app: ActorSystem) {
|
|||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newBalancingDispatcher(name: String) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(app, name, app.AkkaConfig.DispatcherThroughput,
|
||||
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(prerequisites, name, settings.DispatcherThroughput,
|
||||
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -132,7 +147,7 @@ class Dispatchers(val app: ActorSystem) {
|
|||
*/
|
||||
def newBalancingDispatcher(name: String, throughput: Int) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
new BalancingDispatcher(prerequisites, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -141,7 +156,7 @@ class Dispatchers(val app: ActorSystem) {
|
|||
*/
|
||||
def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
new BalancingDispatcher(prerequisites, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
|
||||
|
|
@ -150,13 +165,13 @@ class Dispatchers(val app: ActorSystem) {
|
|||
*/
|
||||
def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app))
|
||||
new BalancingDispatcher(prerequisites, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig())
|
||||
/**
|
||||
* Utility function that tries to load the specified dispatcher config from the akka.conf
|
||||
* or else use the supplied default dispatcher
|
||||
*/
|
||||
def fromConfig(key: String, default: ⇒ MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher =
|
||||
app.config getSection key flatMap from getOrElse default
|
||||
settings.config getSection key flatMap from getOrElse default
|
||||
|
||||
/*
|
||||
* Creates of obtains a dispatcher from a ConfigMap according to the format below
|
||||
|
|
@ -181,9 +196,8 @@ class Dispatchers(val app: ActorSystem) {
|
|||
*/
|
||||
def from(cfg: Configuration): Option[MessageDispatcher] = {
|
||||
cfg.getString("type") flatMap {
|
||||
case "Dispatcher" ⇒ Some(new DispatcherConfigurator(app))
|
||||
case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(app))
|
||||
case "GlobalDispatcher" ⇒ None //TODO FIXME remove this
|
||||
case "Dispatcher" ⇒ Some(new DispatcherConfigurator())
|
||||
case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator())
|
||||
case fqn ⇒
|
||||
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
|
||||
case Right(clazz) ⇒
|
||||
|
|
@ -197,31 +211,35 @@ class Dispatchers(val app: ActorSystem) {
|
|||
throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn, exception)
|
||||
}
|
||||
} map {
|
||||
_ configure cfg
|
||||
_.configure(cfg, settings, prerequisites)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class DispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) {
|
||||
def configure(config: Configuration): MessageDispatcher = {
|
||||
configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(app,
|
||||
class DispatcherConfigurator() extends MessageDispatcherConfigurator() {
|
||||
def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
|
||||
configureThreadPool(config,
|
||||
settings,
|
||||
threadPoolConfig ⇒ new Dispatcher(prerequisites,
|
||||
config.getString("name", newUuid.toString),
|
||||
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput),
|
||||
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
|
||||
mailboxType(config),
|
||||
config.getInt("throughput", settings.DispatcherThroughput),
|
||||
config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
|
||||
mailboxType(config, settings),
|
||||
threadPoolConfig,
|
||||
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
|
||||
settings.DispatcherDefaultShutdown.toMillis)).build
|
||||
}
|
||||
}
|
||||
|
||||
class BalancingDispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) {
|
||||
def configure(config: Configuration): MessageDispatcher = {
|
||||
configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(app,
|
||||
class BalancingDispatcherConfigurator() extends MessageDispatcherConfigurator() {
|
||||
def configure(config: Configuration, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
|
||||
configureThreadPool(config,
|
||||
settings,
|
||||
threadPoolConfig ⇒ new BalancingDispatcher(prerequisites,
|
||||
config.getString("name", newUuid.toString),
|
||||
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput),
|
||||
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
|
||||
mailboxType(config),
|
||||
config.getInt("throughput", settings.DispatcherThroughput),
|
||||
config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt),
|
||||
mailboxType(config, settings),
|
||||
threadPoolConfig,
|
||||
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
|
||||
settings.DispatcherDefaultShutdown.toMillis)).build
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -262,7 +262,7 @@ object Future {
|
|||
result completeWithResult currentValue
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
|
||||
result completeWithException e
|
||||
} finally {
|
||||
results.clear
|
||||
|
|
@ -631,7 +631,7 @@ sealed trait Future[+T] extends japi.Future[T] {
|
|||
Right(f(res))
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
|
||||
Left(e)
|
||||
})
|
||||
}
|
||||
|
|
@ -683,7 +683,7 @@ sealed trait Future[+T] extends japi.Future[T] {
|
|||
future.completeWith(f(r))
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
|
||||
future complete Left(e)
|
||||
}
|
||||
}
|
||||
|
|
@ -716,7 +716,7 @@ sealed trait Future[+T] extends japi.Future[T] {
|
|||
if (p(res)) r else Left(new MatchError(res))
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
|
||||
Left(e)
|
||||
})
|
||||
}
|
||||
|
|
@ -788,7 +788,7 @@ trait Promise[T] extends Future[T] {
|
|||
fr completeWith cont(f)
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
|
||||
fr completeWithException e
|
||||
}
|
||||
}
|
||||
|
|
@ -802,7 +802,7 @@ trait Promise[T] extends Future[T] {
|
|||
fr completeWith cont(f)
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage))
|
||||
dispatcher.prerequisites.eventStream.publish(Error(e, this, e.getMessage))
|
||||
fr completeWithException e
|
||||
}
|
||||
}
|
||||
|
|
@ -956,12 +956,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
|
|||
val runnable = new Runnable {
|
||||
def run() {
|
||||
if (!isCompleted) {
|
||||
if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
|
||||
if (!isExpired) dispatcher.prerequisites.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
|
||||
else func(DefaultPromise.this)
|
||||
}
|
||||
}
|
||||
}
|
||||
val timeoutFuture = dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
|
||||
val timeoutFuture = dispatcher.prerequisites.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
|
||||
onComplete(_ ⇒ timeoutFuture.cancel())
|
||||
false
|
||||
} else true
|
||||
|
|
@ -983,18 +983,18 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
|
|||
val runnable = new Runnable {
|
||||
def run() {
|
||||
if (!isCompleted) {
|
||||
if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
|
||||
if (!isExpired) dispatcher.prerequisites.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
|
||||
else promise complete (try { Right(fallback) } catch { case e ⇒ Left(e) })
|
||||
}
|
||||
}
|
||||
}
|
||||
dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
|
||||
dispatcher.prerequisites.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
|
||||
promise
|
||||
}
|
||||
} else this
|
||||
|
||||
private def notifyCompleted(func: Future[T] ⇒ Unit) {
|
||||
try { func(this) } catch { case e ⇒ dispatcher.app.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really?
|
||||
try { func(this) } catch { case e ⇒ dispatcher.prerequisites.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really?
|
||||
}
|
||||
|
||||
@inline
|
||||
|
|
|
|||
|
|
@ -187,7 +187,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
|
|||
}
|
||||
} catch {
|
||||
case e ⇒
|
||||
actor.app.eventStream.publish(Error(e, actor.self, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
|
||||
actor.system.eventStream.publish(Error(e, actor.self, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
|
||||
throw e
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,14 +7,27 @@ package akka.dispatch
|
|||
import java.util.concurrent.atomic.AtomicReference
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.ActorSystem
|
||||
import akka.event.EventStream
|
||||
import akka.actor.Scheduler
|
||||
|
||||
/**
|
||||
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long)
|
||||
extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) {
|
||||
class PinnedDispatcher(
|
||||
_prerequisites: DispatcherPrerequisites,
|
||||
_actor: ActorCell,
|
||||
_name: String,
|
||||
_mailboxType: MailboxType,
|
||||
_timeoutMs: Long)
|
||||
extends Dispatcher(_prerequisites,
|
||||
_name,
|
||||
Int.MaxValue,
|
||||
-1,
|
||||
_mailboxType,
|
||||
ThreadPoolConfig(allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1),
|
||||
_timeoutMs) {
|
||||
|
||||
@volatile
|
||||
protected[akka] var owner: ActorCell = _actor
|
||||
|
|
@ -33,7 +46,3 @@ class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mai
|
|||
}
|
||||
}
|
||||
|
||||
object PinnedDispatcher {
|
||||
def oneThread(app: ActorSystem): ThreadPoolConfig = ThreadPoolConfig(app, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.util.Duration
|
|||
import akka.event.Logging.{ Warning, Error }
|
||||
import akka.actor.ActorSystem
|
||||
import java.util.concurrent._
|
||||
import akka.event.EventStream
|
||||
|
||||
object ThreadPoolConfig {
|
||||
type Bounds = Int
|
||||
|
|
@ -63,8 +64,7 @@ trait ExecutorServiceFactoryProvider {
|
|||
/**
|
||||
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
|
||||
*/
|
||||
case class ThreadPoolConfig(app: ActorSystem,
|
||||
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
|
||||
case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
|
||||
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
|
||||
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
|
||||
threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,
|
||||
|
|
|
|||
|
|
@ -3,9 +3,7 @@
|
|||
*/
|
||||
package akka.event
|
||||
|
||||
import akka.actor.{ ActorRef, Actor, Props }
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.Terminated
|
||||
import akka.actor.{ ActorRef, Actor, Props, ActorSystemImpl, Terminated }
|
||||
import akka.util.Subclassification
|
||||
|
||||
class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification {
|
||||
|
|
@ -41,8 +39,8 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas
|
|||
super.unsubscribe(subscriber)
|
||||
}
|
||||
|
||||
def start(app: ActorSystem) {
|
||||
reaper = app.systemActorOf(Props(new Actor {
|
||||
def start(system: ActorSystemImpl) {
|
||||
reaper = system.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case ref: ActorRef ⇒ watch(ref)
|
||||
case Terminated(ref) ⇒ unsubscribe(ref)
|
||||
|
|
|
|||
|
|
@ -3,9 +3,9 @@
|
|||
*/
|
||||
package akka.event
|
||||
|
||||
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, simpleName }
|
||||
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, ActorSystemImpl, simpleName }
|
||||
import akka.AkkaException
|
||||
import akka.actor.ActorSystem.AkkaConfig
|
||||
import akka.actor.ActorSystem.Settings
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.config.ConfigurationException
|
||||
import akka.util.ReentrantGuard
|
||||
|
|
@ -13,6 +13,7 @@ import akka.util.duration._
|
|||
import akka.actor.Timeout
|
||||
import akka.dispatch.FutureTimeoutException
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor.ActorRefProvider
|
||||
|
||||
/**
|
||||
* This trait brings log level handling to the EventStream: it reads the log
|
||||
|
|
@ -65,7 +66,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
_logLevel = level
|
||||
}
|
||||
|
||||
private[akka] def startStdoutLogger(config: AkkaConfig) {
|
||||
private[akka] def startStdoutLogger(config: Settings) {
|
||||
val level = levelFor(config.StdoutLogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.StdoutLogLevel))
|
||||
ErrorLevel
|
||||
|
|
@ -78,13 +79,13 @@ trait LoggingBus extends ActorEventBus {
|
|||
publish(Info(this, "StandardOutLogger started"))
|
||||
}
|
||||
|
||||
private[akka] def startDefaultLoggers(app: ActorSystem, config: AkkaConfig) {
|
||||
val level = levelFor(config.LogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel))
|
||||
private[akka] def startDefaultLoggers(system: ActorSystemImpl) {
|
||||
val level = levelFor(system.settings.LogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + system.settings.LogLevel))
|
||||
ErrorLevel
|
||||
}
|
||||
try {
|
||||
val defaultLoggers = config.EventHandlers match {
|
||||
val defaultLoggers = system.settings.EventHandlers match {
|
||||
case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil
|
||||
case loggers ⇒ loggers
|
||||
}
|
||||
|
|
@ -94,7 +95,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
} yield {
|
||||
try {
|
||||
ReflectiveAccess.getClassFor[Actor](loggerName) match {
|
||||
case Right(actorClass) ⇒ addLogger(app, actorClass, level)
|
||||
case Right(actorClass) ⇒ addLogger(system, actorClass, level)
|
||||
case Left(exception) ⇒ throw exception
|
||||
}
|
||||
} catch {
|
||||
|
|
@ -137,9 +138,9 @@ trait LoggingBus extends ActorEventBus {
|
|||
publish(Info(this, "all default loggers stopped"))
|
||||
}
|
||||
|
||||
private def addLogger(app: ActorSystem, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
|
||||
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
|
||||
val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz)
|
||||
val actor = app.systemActorOf(Props(clazz), name)
|
||||
val actor = system.systemActorOf(Props(clazz), name)
|
||||
implicit val timeout = Timeout(3 seconds)
|
||||
val response = try actor ? InitializeLogger(this) get catch {
|
||||
case _: FutureTimeoutException ⇒
|
||||
|
|
@ -236,12 +237,12 @@ object Logging {
|
|||
* Obtain LoggingAdapter for the given application and source object. The
|
||||
* source object is used to identify the source of this logging channel.
|
||||
*/
|
||||
def apply(app: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(app.eventStream, source)
|
||||
def apply(system: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(system.eventStream, source)
|
||||
/**
|
||||
* Java API: Obtain LoggingAdapter for the given application and source object. The
|
||||
* source object is used to identify the source of this logging channel.
|
||||
*/
|
||||
def getLogger(app: ActorSystem, source: AnyRef): LoggingAdapter = apply(app, source)
|
||||
def getLogger(system: ActorSystem, source: AnyRef): LoggingAdapter = apply(system, source)
|
||||
/**
|
||||
* Obtain LoggingAdapter for the given event bus and source object. The
|
||||
* source object is used to identify the source of this logging channel.
|
||||
|
|
|
|||
|
|
@ -128,7 +128,7 @@ case class CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorExcept
|
|||
override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter)
|
||||
}
|
||||
|
||||
abstract class RemoteSupport(val app: ActorSystem) {
|
||||
abstract class RemoteSupport(val system: ActorSystem) {
|
||||
/**
|
||||
* Shuts down the remoting
|
||||
*/
|
||||
|
|
@ -162,7 +162,7 @@ abstract class RemoteSupport(val app: ActorSystem) {
|
|||
recipient: ActorRef,
|
||||
loader: Option[ClassLoader]): Unit
|
||||
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = app.eventStream.publish(message)
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = system.eventStream.publish(message)
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
|
|
|||
|
|
@ -155,7 +155,7 @@ object Routing {
|
|||
/**
|
||||
* An Abstract convenience implementation for building an ActorReference that uses a Router.
|
||||
*/
|
||||
abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val props: RoutedProps) extends MinimalActorRef {
|
||||
abstract private[akka] class AbstractRoutedActorRef(val system: ActorSystem, val props: RoutedProps) extends MinimalActorRef {
|
||||
val router = props.routerFactory()
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = router.route(message)(sender)
|
||||
|
|
@ -167,7 +167,7 @@ abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val pr
|
|||
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
|
||||
* on (or more) of these actors.
|
||||
*/
|
||||
private[akka] class RoutedActorRef(app: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(app, routedProps) {
|
||||
private[akka] class RoutedActorRef(system: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(system, routedProps) {
|
||||
|
||||
val path = supervisor.path / name
|
||||
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.serialization
|
|||
|
||||
import akka.AkkaException
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import scala.util.DynamicVariable
|
||||
import akka.remote.RemoteSupport
|
||||
|
||||
|
|
@ -16,7 +16,7 @@ case class NoSerializerFoundException(m: String) extends AkkaException(m)
|
|||
* Serialization module. Contains methods for serialization and deserialization as well as
|
||||
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
|
||||
*/
|
||||
class Serialization(val app: ActorSystem) {
|
||||
class Serialization(val system: ActorSystemImpl) {
|
||||
|
||||
//TODO document me
|
||||
def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
|
||||
|
|
@ -28,7 +28,7 @@ class Serialization(val app: ActorSystem) {
|
|||
clazz: Class[_],
|
||||
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
|
||||
try {
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.system.withValue(system) {
|
||||
Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader))
|
||||
}
|
||||
} catch { case e: Exception ⇒ Left(e) }
|
||||
|
|
@ -70,7 +70,7 @@ class Serialization(val app: ActorSystem) {
|
|||
* But "default" can be overridden in config
|
||||
*/
|
||||
val serializers: Map[String, Serializer] =
|
||||
app.config.getSection("akka.actor.serializers")
|
||||
system.settings.config.getSection("akka.actor.serializers")
|
||||
.map(_.map)
|
||||
.getOrElse(Map())
|
||||
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) {
|
||||
|
|
@ -81,7 +81,7 @@ class Serialization(val app: ActorSystem) {
|
|||
/**
|
||||
* bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used
|
||||
*/
|
||||
val bindings: Map[String, String] = app.config.getSection("akka.actor.serialization-bindings") map {
|
||||
val bindings: Map[String, String] = system.settings.config.getSection("akka.actor.serialization-bindings") map {
|
||||
_.map.foldLeft(Map[String, String]()) {
|
||||
case (result, (k: String, vs: List[_])) ⇒ result ++ (vs collect { case v: String ⇒ (v, k) }) //All keys which are lists, take the Strings from them and Map them
|
||||
case (result, _) ⇒ result //For any other values, just skip them, TODO: print out warnings?
|
||||
|
|
@ -102,6 +102,6 @@ class Serialization(val app: ActorSystem) {
|
|||
|
||||
object Serialization {
|
||||
// TODO ensure that these are always set (i.e. withValue()) when doing deserialization
|
||||
val app = new DynamicVariable[ActorSystem](null)
|
||||
val system = new DynamicVariable[ActorSystemImpl](null)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import akka.actor.ActorSystem
|
|||
/*
|
||||
* This class is responsible for booting up a stack of bundles and then shutting them down
|
||||
*/
|
||||
class AkkaLoader(app: ActorSystem) {
|
||||
class AkkaLoader(system: ActorSystem) {
|
||||
private val hasBooted = new Switch(false)
|
||||
|
||||
@volatile
|
||||
|
|
|
|||
|
|
@ -278,7 +278,7 @@ abstract class Duration extends Serializable {
|
|||
def /(other: Duration): Double
|
||||
def unary_- : Duration
|
||||
def finite_? : Boolean
|
||||
def dilated(implicit app: ActorSystem): Duration = this * app.AkkaConfig.TestTimeFactor
|
||||
def dilated(implicit system: ActorSystem): Duration = this * system.settings.TestTimeFactor
|
||||
def min(other: Duration): Duration = if (this < other) this else other
|
||||
def max(other: Duration): Duration = if (this > other) this else other
|
||||
def sleep(): Unit = Thread.sleep(toMillis)
|
||||
|
|
|
|||
|
|
@ -18,20 +18,20 @@ object JMX {
|
|||
def nameFor(hostname: String, service: String, bean: String): ObjectName =
|
||||
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
|
||||
|
||||
def register(name: ObjectName, mbean: AnyRef)(implicit app: ActorSystem): Option[ObjectInstance] = try {
|
||||
def register(name: ObjectName, mbean: AnyRef)(implicit system: ActorSystem): Option[ObjectInstance] = try {
|
||||
Some(mbeanServer.registerMBean(mbean, name))
|
||||
} catch {
|
||||
case e: InstanceAlreadyExistsException ⇒
|
||||
Some(mbeanServer.getObjectInstance(name))
|
||||
case e: Exception ⇒
|
||||
app.eventStream.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean)))
|
||||
system.eventStream.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean)))
|
||||
None
|
||||
}
|
||||
|
||||
def unregister(mbean: ObjectName)(implicit app: ActorSystem) = try {
|
||||
def unregister(mbean: ObjectName)(implicit system: ActorSystem) = try {
|
||||
mbeanServer.unregisterMBean(mbean)
|
||||
} catch {
|
||||
case e: InstanceNotFoundException ⇒ {}
|
||||
case e: Exception ⇒ app.eventStream.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean)))
|
||||
case e: Exception ⇒ system.eventStream.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean)))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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é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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -58,7 +58,7 @@ trait CamelService extends Bootable {
|
|||
* Starts this CamelService.
|
||||
*/
|
||||
def start: CamelService = {
|
||||
// Only init and start if not already done by app
|
||||
// Only init and start if not already done by system
|
||||
if (!CamelContextManager.initialized) CamelContextManager.init
|
||||
if (!CamelContextManager.started) CamelContextManager.start
|
||||
|
||||
|
|
|
|||
|
|
@ -35,7 +35,7 @@ trait Consumer { this: Actor ⇒
|
|||
|
||||
/**
|
||||
* Determines whether one-way communications between an endpoint and this consumer actor
|
||||
* should be auto-acknowledged or app-acknowledged.
|
||||
* should be auto-acknowledged or system-acknowledged.
|
||||
*/
|
||||
def autoack = true
|
||||
|
||||
|
|
@ -79,7 +79,7 @@ abstract class UntypedConsumerActor extends UntypedActor with Consumer {
|
|||
|
||||
/**
|
||||
* Determines whether one-way communications between an endpoint and this consumer actor
|
||||
* should be auto-acknowledged or app-acknowledged.
|
||||
* should be auto-acknowledged or system-acknowledged.
|
||||
*/
|
||||
def isAutoack() = super.autoack
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -50,7 +50,7 @@ trait ProducerSupport { this: Actor ⇒
|
|||
/**
|
||||
* Returns the names of message headers to copy from a request message to a response message.
|
||||
* By default only the Message.MessageExchangeId is copied. Applications may override this to
|
||||
* define an app-specific set of message headers to copy.
|
||||
* define an system-specific set of message headers to copy.
|
||||
*/
|
||||
def headersToCopy: Set[String] = headersToCopyDefault
|
||||
|
||||
|
|
|
|||
|
|
@ -138,15 +138,15 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
|
||||
"An non auto-acknowledging consumer" when {
|
||||
"started" must {
|
||||
"must support acknowledgements on app level" in {
|
||||
"must support acknowledgements on system level" in {
|
||||
|
||||
var consumer: ActorRef = null
|
||||
|
||||
service.awaitEndpointActivation(1) {
|
||||
consumer = actorOf(new TestAckConsumer("direct:app-ack-test"))
|
||||
consumer = actorOf(new TestAckConsumer("direct:system-ack-test"))
|
||||
} must be(true)
|
||||
|
||||
val endpoint = mandatoryContext.getEndpoint("direct:app-ack-test", classOf[DirectEndpoint])
|
||||
val endpoint = mandatoryContext.getEndpoint("direct:system-ack-test", classOf[DirectEndpoint])
|
||||
val producer = endpoint.createProducer.asInstanceOf[AsyncProcessor]
|
||||
val exchange = endpoint.createExchange
|
||||
|
||||
|
|
|
|||
|
|
@ -9,13 +9,13 @@ import java.io.File
|
|||
|
||||
/*
|
||||
A simple use of BookKeeper is to implement a write-ahead transaction log. A server maintains an in-memory data structure
|
||||
(with periodic snapshots for example) and logs changes to that structure before it applies the change. The app
|
||||
(with periodic snapshots for example) and logs changes to that structure before it applies the change. The system
|
||||
server creates a ledger at startup and store the ledger id and password in a well known place (ZooKeeper maybe). When
|
||||
it needs to make a change, the server adds an entry with the change information to a ledger and apply the change when
|
||||
BookKeeper adds the entry successfully. The server can even use asyncAddEntry to queue up many changes for high change
|
||||
throughput. BooKeeper meticulously logs the changes in order and call the completion functions in order.
|
||||
|
||||
When the app server dies, a backup server will come online, get the last snapshot and then it will open the
|
||||
When the system server dies, a backup server will come online, get the last snapshot and then it will open the
|
||||
ledger of the old server and read all the entries from the time the snapshot was taken. (Since it doesn't know the last
|
||||
entry number it will use MAX_INTEGER). Once all the entries have been processed, it will close the ledger and start a
|
||||
new one for its use.
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import System.{currentTimeMillis => now}
|
|||
import java.util.concurrent.CountDownLatch
|
||||
//#imports
|
||||
|
||||
//#app
|
||||
//#system
|
||||
object Pi extends App {
|
||||
|
||||
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
|
||||
|
|
@ -127,5 +127,5 @@ object Pi extends App {
|
|||
latch.await()
|
||||
}
|
||||
}
|
||||
//#app
|
||||
//#system
|
||||
|
||||
|
|
|
|||
|
|
@ -11,10 +11,10 @@
|
|||
// import java.util.concurrent.CountDownLatch
|
||||
// //#imports
|
||||
|
||||
// //#app
|
||||
// //#system
|
||||
// object Pi extends App {
|
||||
|
||||
// val app = ActorSystem()
|
||||
// val system = ActorSystem()
|
||||
|
||||
// calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
|
||||
|
||||
|
|
@ -66,10 +66,10 @@
|
|||
|
||||
// //#create-workers
|
||||
// // create the workers
|
||||
// val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker])
|
||||
// val workers = Vector.fill(nrOfWorkers)(system.actorOf[Worker])
|
||||
|
||||
// // wrap them with a load-balancing router
|
||||
// val router = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
|
||||
// val router = system.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
|
||||
// //#create-workers
|
||||
|
||||
// //#master-receive
|
||||
|
|
@ -119,7 +119,7 @@
|
|||
// val latch = new CountDownLatch(1)
|
||||
|
||||
// // create the master
|
||||
// val master = app.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
|
||||
// val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
|
||||
|
||||
// // start the calculation
|
||||
// master ! Calculate
|
||||
|
|
@ -128,5 +128,5 @@
|
|||
// latch.await()
|
||||
// }
|
||||
// }
|
||||
// //#app
|
||||
// //#system
|
||||
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ import akka.config.Configuration
|
|||
|
||||
//#my-actor
|
||||
class MyActor extends Actor {
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
def receive = {
|
||||
case "test" ⇒ log.info("received test")
|
||||
case _ ⇒ log.info("received unknown message")
|
||||
|
|
@ -36,8 +36,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
|
|||
case e: Logging.Info ⇒ true
|
||||
case _ ⇒ false
|
||||
}
|
||||
app.eventStream.publish(TestEvent.Mute(filter))
|
||||
app.eventStream.subscribe(testActor, classOf[Logging.Info])
|
||||
system.eventStream.publish(TestEvent.Mute(filter))
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Info])
|
||||
|
||||
myActor ! "test"
|
||||
expectMsgPF(1 second) { case Logging.Info(_, "received test") ⇒ true }
|
||||
|
|
@ -45,8 +45,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
|
|||
myActor ! "unknown"
|
||||
expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") ⇒ true }
|
||||
|
||||
app.eventStream.unsubscribe(testActor)
|
||||
app.eventStream.publish(TestEvent.UnMute(filter))
|
||||
system.eventStream.unsubscribe(testActor)
|
||||
system.eventStream.publish(TestEvent.UnMute(filter))
|
||||
|
||||
myActor.stop()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -20,15 +20,15 @@ class BeanstalkBasedMailboxException(message: String) extends AkkaException(mess
|
|||
*/
|
||||
class BeanstalkBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
|
||||
|
||||
val hostname = app.config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0")
|
||||
val port = app.config.getInt("akka.actor.mailbox.beanstalk.port", 11300)
|
||||
def defaultTimeUnit = app.AkkaConfig.DefaultTimeUnit
|
||||
val reconnectWindow = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), defaultTimeUnit).toSeconds.toInt
|
||||
val messageSubmitDelay = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.message-submit-delay", 0), defaultTimeUnit).toSeconds.toInt
|
||||
val messageSubmitTimeout = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.message-submit-timeout", 5), defaultTimeUnit).toSeconds.toInt
|
||||
val messageTimeToLive = Duration(app.config.getInt("akka.actor.mailbox.beanstalk.message-time-to-live", 120), defaultTimeUnit).toSeconds.toInt
|
||||
val hostname = system.settings.config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0")
|
||||
val port = system.settings.config.getInt("akka.actor.mailbox.beanstalk.port", 11300)
|
||||
def defaultTimeUnit = system.settings.DefaultTimeUnit
|
||||
val reconnectWindow = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), defaultTimeUnit).toSeconds.toInt
|
||||
val messageSubmitDelay = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.message-submit-delay", 0), defaultTimeUnit).toSeconds.toInt
|
||||
val messageSubmitTimeout = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.message-submit-timeout", 5), defaultTimeUnit).toSeconds.toInt
|
||||
val messageTimeToLive = Duration(system.settings.config.getInt("akka.actor.mailbox.beanstalk.message-time-to-live", 120), defaultTimeUnit).toSeconds.toInt
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
private val queue = new ThreadLocal[Client] { override def initialValue = connect(name) }
|
||||
|
||||
|
|
|
|||
|
|
@ -19,13 +19,13 @@ object FileBasedMailbox {
|
|||
|
||||
class FileBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
val queuePath = FileBasedMailbox.queuePath(owner.app.config)
|
||||
val queuePath = FileBasedMailbox.queuePath(owner.system.settings.config)
|
||||
|
||||
private val queue = try {
|
||||
try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case e ⇒ {} }
|
||||
val queue = new filequeue.PersistentQueue(queuePath, name, owner.app.config, log)
|
||||
val queue = new filequeue.PersistentQueue(queuePath, name, owner.system.settings.config, log)
|
||||
queue.setup // replays journal
|
||||
queue.discardExpired
|
||||
queue
|
||||
|
|
|
|||
|
|
@ -142,11 +142,11 @@ object QDumper {
|
|||
System.exit(0)
|
||||
}
|
||||
|
||||
val app = ActorSystem()
|
||||
val system = ActorSystem()
|
||||
|
||||
for (filename ← filenames) {
|
||||
println("Queue: " + filename)
|
||||
new QueueDumper(filename, app.log)()
|
||||
new QueueDumper(filename, system.log)()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ import org.apache.commons.io.FileUtils
|
|||
class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileDurableMailboxType) {
|
||||
|
||||
def clean {
|
||||
val queuePath = FileBasedMailbox.queuePath(app.config)
|
||||
val queuePath = FileBasedMailbox.queuePath(system.settings.config)
|
||||
FileUtils.deleteDirectory(new java.io.File(queuePath))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ class DurableMailboxException private[akka] (message: String, cause: Throwable)
|
|||
abstract class DurableMailbox(owner: ActorCell) extends Mailbox(owner) with DefaultSystemMessageQueue {
|
||||
import DurableExecutableMailboxConfig._
|
||||
|
||||
def app = owner.app
|
||||
def system = owner.system
|
||||
def ownerPath = owner.self.path
|
||||
val ownerPathString = ownerPath.path.mkString("/")
|
||||
val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
|
||||
|
|
@ -54,11 +54,11 @@ trait DurableMessageSerialization {
|
|||
def serialize(durableMessage: Envelope): Array[Byte] = {
|
||||
|
||||
def serializeActorRef(ref: ActorRef): ActorRefProtocol = {
|
||||
val serRef = owner.app.provider.serialize(ref)
|
||||
val serRef = owner.system.provider.serialize(ref)
|
||||
ActorRefProtocol.newBuilder.setPath(serRef.path).setHost(serRef.hostname).setPort(serRef.port).build
|
||||
}
|
||||
|
||||
val message = MessageSerializer.serialize(owner.app, durableMessage.message.asInstanceOf[AnyRef])
|
||||
val message = MessageSerializer.serialize(owner.system, durableMessage.message.asInstanceOf[AnyRef])
|
||||
val builder = RemoteMessageProtocol.newBuilder
|
||||
.setMessage(message)
|
||||
.setRecipient(serializeActorRef(owner.self))
|
||||
|
|
@ -71,11 +71,11 @@ trait DurableMessageSerialization {
|
|||
|
||||
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = {
|
||||
val serRef = SerializedActorRef(refProtocol.getHost, refProtocol.getPort, refProtocol.getPath)
|
||||
owner.app.provider.deserialize(serRef).getOrElse(owner.app.deadLetters)
|
||||
owner.system.provider.deserialize(serRef).getOrElse(owner.system.deadLetters)
|
||||
}
|
||||
|
||||
val durableMessage = RemoteMessageProtocol.parseFrom(bytes)
|
||||
val message = MessageSerializer.deserialize(owner.app, durableMessage.getMessage)
|
||||
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage)
|
||||
val sender = deserializeActorRef(durableMessage.getSender)
|
||||
|
||||
new Envelope(message, sender)
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ object DurableMailboxSpecActorFactory {
|
|||
abstract class DurableMailboxSpec(val backendName: String, val mailboxType: DurableMailboxType) extends AkkaSpec with BeforeAndAfterEach {
|
||||
import DurableMailboxSpecActorFactory._
|
||||
|
||||
implicit val dispatcher = new Dispatchers(app).newDispatcher(backendName, throughput = 1, mailboxType = mailboxType).build
|
||||
implicit val dispatcher = system.dispatcherFactory.newDispatcher(backendName, throughput = 1, mailboxType = mailboxType).build
|
||||
|
||||
def createMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef =
|
||||
actorOf(Props(new MailboxTestActor).withDispatcher(dispatcher))
|
||||
|
|
|
|||
|
|
@ -17,9 +17,11 @@ import org.bson.DefaultBSONSerializer
|
|||
import akka.actor.SerializedActorRef
|
||||
import akka.remote.RemoteProtocol.MessageProtocol
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
|
||||
class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
|
||||
val systemImpl = system.asInstanceOf[ActorSystemImpl]
|
||||
|
||||
protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = {
|
||||
|
||||
|
|
@ -28,7 +30,7 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
|
|||
b += "_id" -> msg._id
|
||||
b += "ownerPath" -> msg.ownerPath
|
||||
|
||||
val sender = app.provider.serialize(msg.sender)
|
||||
val sender = systemImpl.provider.serialize(msg.sender)
|
||||
b += "senderPath" -> sender.path
|
||||
b += "senderHostname" -> sender.hostname
|
||||
b += "senderPort" -> sender.port
|
||||
|
|
@ -37,10 +39,10 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
|
|||
* TODO - Figure out a way for custom serialization of the message instance
|
||||
* TODO - Test if a serializer is registered for the message and if not, use toByteString
|
||||
*/
|
||||
val msgData = MessageSerializer.serialize(app, msg.message.asInstanceOf[AnyRef])
|
||||
val msgData = MessageSerializer.serialize(system, msg.message.asInstanceOf[AnyRef])
|
||||
b += "message" -> new org.bson.types.Binary(0, msgData.toByteArray)
|
||||
val doc = b.result
|
||||
app.log.debug("Serialized Document: {}", doc)
|
||||
system.log.debug("Serialized Document: {}", doc)
|
||||
serializer.putObject(doc)
|
||||
}
|
||||
|
||||
|
|
@ -68,15 +70,15 @@ class BSONSerializableMailbox(app: ActorSystem) extends SerializableBSONObject[M
|
|||
val deserializer = new DefaultBSONDeserializer
|
||||
// TODO - Skip the whole doc step for performance, fun, and profit! (Needs Salat / custom Deser)
|
||||
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
|
||||
app.log.debug("Deserializing a durable message from MongoDB: {}", doc)
|
||||
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
|
||||
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
|
||||
val msg = MessageSerializer.deserialize(app, msgData)
|
||||
val msg = MessageSerializer.deserialize(system, msgData)
|
||||
val ownerPath = doc.as[String]("ownerPath")
|
||||
val senderPath = doc.as[String]("senderPath")
|
||||
val senderHostname = doc.as[String]("senderHostname")
|
||||
val senderPort = doc.as[Int]("senderPort")
|
||||
val sender = app.provider.deserialize(SerializedActorRef(senderHostname, senderPort, senderPath)).
|
||||
getOrElse(app.deadLetters)
|
||||
val sender = systemImpl.provider.deserialize(SerializedActorRef(senderHostname, senderPort, senderPath)).
|
||||
getOrElse(system.deadLetters)
|
||||
|
||||
MongoDurableMessage(ownerPath, msg, sender)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -28,17 +28,17 @@ class MongoBasedMailboxException(message: String) extends AkkaException(message)
|
|||
*/
|
||||
class MongoBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) {
|
||||
// this implicit object provides the context for reading/writing things as MongoDurableMessage
|
||||
implicit val mailboxBSONSer = new BSONSerializableMailbox(app)
|
||||
implicit val mailboxBSONSer = new BSONSerializableMailbox(system)
|
||||
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
|
||||
|
||||
val URI_CONFIG_KEY = "akka.actor.mailbox.mongodb.uri"
|
||||
val WRITE_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.write"
|
||||
val READ_TIMEOUT_KEY = "akka.actor.mailbox.mongodb.timeout.read"
|
||||
val mongoURI = app.config.getString(URI_CONFIG_KEY)
|
||||
val writeTimeout = app.config.getInt(WRITE_TIMEOUT_KEY, 3000)
|
||||
val readTimeout = app.config.getInt(READ_TIMEOUT_KEY, 3000)
|
||||
val mongoURI = system.settings.config.getString(URI_CONFIG_KEY)
|
||||
val writeTimeout = system.settings.config.getInt(WRITE_TIMEOUT_KEY, 3000)
|
||||
val readTimeout = system.settings.config.getInt(READ_TIMEOUT_KEY, 3000)
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
@volatile
|
||||
private var mongo = connect()
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ class RedisBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with
|
|||
@volatile
|
||||
private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
def enqueue(receiver: ActorRef, envelope: Envelope) {
|
||||
log.debug("ENQUEUING message in redis-based mailbox [%s]".format(envelope))
|
||||
|
|
@ -58,8 +58,8 @@ class RedisBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with
|
|||
|
||||
private[akka] def connect() = {
|
||||
new RedisClientPool(
|
||||
app.config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"),
|
||||
app.config.getInt("akka.actor.mailbox.redis.port", 6379))
|
||||
system.settings.config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"),
|
||||
system.settings.config.getInt("akka.actor.mailbox.redis.port", 6379))
|
||||
}
|
||||
|
||||
private def withErrorHandling[T](body: ⇒ T): T = {
|
||||
|
|
|
|||
|
|
@ -21,16 +21,16 @@ class ZooKeeperBasedMailboxException(message: String) extends AkkaException(mess
|
|||
*/
|
||||
class ZooKeeperBasedMailbox(val owner: ActorCell) extends DurableMailbox(owner) with DurableMessageSerialization {
|
||||
|
||||
val zkServerAddresses = app.config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181")
|
||||
def defaultTimeUnit = app.AkkaConfig.DefaultTimeUnit
|
||||
val sessionTimeout = Duration(app.config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), defaultTimeUnit).toMillis.toInt
|
||||
val connectionTimeout = Duration(app.config.getInt("akka.actor.mailbox.zookeeper.connection-timeout", 60), defaultTimeUnit).toMillis.toInt
|
||||
val blockingQueue = app.config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true)
|
||||
val zkServerAddresses = system.settings.config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181")
|
||||
def defaultTimeUnit = system.settings.DefaultTimeUnit
|
||||
val sessionTimeout = Duration(system.settings.config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), defaultTimeUnit).toMillis.toInt
|
||||
val connectionTimeout = Duration(system.settings.config.getInt("akka.actor.mailbox.zookeeper.connection-timeout", 60), defaultTimeUnit).toMillis.toInt
|
||||
val blockingQueue = system.settings.config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true)
|
||||
|
||||
val queueNode = "/queues"
|
||||
val queuePathTemplate = queueNode + "/%s"
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
private val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout)
|
||||
private val queue = new ZooKeeperQueue[Array[Byte]](zkClient, queuePathTemplate.format(name), blockingQueue)
|
||||
|
|
|
|||
|
|
@ -14,13 +14,13 @@ import javax.servlet.{ ServletContextListener, ServletContextEvent }
|
|||
/**
|
||||
* This class can be added to web.xml mappings as a listener to start and postStop Akka.
|
||||
*
|
||||
* <web-app>
|
||||
* <web-system>
|
||||
* ...
|
||||
* <listener>
|
||||
* <listener-class>akka.servlet.Initializer</listener-class>
|
||||
* </listener>
|
||||
* ...
|
||||
* </web-app>
|
||||
* </web-system>
|
||||
*/
|
||||
class Initializer extends ServletContextListener {
|
||||
lazy val loader = new AkkaLoader
|
||||
|
|
|
|||
|
|
@ -25,10 +25,10 @@ import akka.actor.ActorSystem
|
|||
*/
|
||||
class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 1000) {
|
||||
|
||||
def this(app: ActorSystem) {
|
||||
def this(system: ActorSystem) {
|
||||
this(
|
||||
app.config.getInt("akka.remote.failure-detector.theshold", 8),
|
||||
app.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
|
||||
system.settings.config.getInt("akka.remote.failure-detector.theshold", 8),
|
||||
system.settings.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
|
||||
}
|
||||
|
||||
private final val PhiFactor = 1.0 / math.log(10.0)
|
||||
|
|
|
|||
|
|
@ -21,26 +21,26 @@ trait BootableRemoteActorService extends Bootable {
|
|||
def settings: RemoteServerSettings
|
||||
|
||||
protected lazy val remoteServerThread = new Thread(new Runnable() {
|
||||
def run = app.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port
|
||||
def run = system.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port
|
||||
}, "Akka RemoteModule Service")
|
||||
|
||||
def startRemoteService() { remoteServerThread.start() }
|
||||
|
||||
abstract override def onLoad() {
|
||||
if (app.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
|
||||
app.eventHandler.info(this, "Initializing Remote Actors Service...")
|
||||
if (system.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) {
|
||||
system.eventHandler.info(this, "Initializing Remote Actors Service...")
|
||||
startRemoteService()
|
||||
app.eventHandler.info(this, "Remote Actors Service initialized")
|
||||
system.eventHandler.info(this, "Remote Actors Service initialized")
|
||||
}
|
||||
super.onLoad()
|
||||
}
|
||||
|
||||
abstract override def onUnload() {
|
||||
app.eventHandler.info(this, "Shutting down Remote Actors Service")
|
||||
system.eventHandler.info(this, "Shutting down Remote Actors Service")
|
||||
|
||||
app.remote.shutdown()
|
||||
system.remote.shutdown()
|
||||
if (remoteServerThread.isAlive) remoteServerThread.join(1000)
|
||||
app.eventHandler.info(this, "Remote Actors Service has been shut down")
|
||||
system.eventHandler.info(this, "Remote Actors Service has been shut down")
|
||||
super.onUnload()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -101,13 +101,13 @@ class Gossiper(remote: Remote) {
|
|||
currentGossip: Gossip,
|
||||
nodeMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener])
|
||||
|
||||
private val app = remote.app
|
||||
private val log = Logging(app, this)
|
||||
private val system = remote.system
|
||||
private val log = Logging(system, this)
|
||||
private val failureDetector = remote.failureDetector
|
||||
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
|
||||
private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[RemoteAddress, ActorRef])
|
||||
private val seeds = Set(address) // FIXME read in list of seeds from config
|
||||
|
||||
private val address = app.address
|
||||
private val address = system.rootPath.remoteAddress
|
||||
private val nodeFingerprint = address.##
|
||||
|
||||
private val random = SecureRandom.getInstance("SHA1PRNG")
|
||||
|
|
@ -122,8 +122,8 @@ class Gossiper(remote: Remote) {
|
|||
|
||||
{
|
||||
// start periodic gossip and cluster scrutinization - default is run them every second with 1/2 second in between
|
||||
app.scheduler schedule (() ⇒ initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
|
||||
app.scheduler schedule (() ⇒ scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
|
||||
system.scheduler schedule (() ⇒ initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
|
||||
system.scheduler schedule (() ⇒ scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -153,7 +153,7 @@ class Gossiper(remote: Remote) {
|
|||
node ← oldAvailableNodes
|
||||
if connectionManager.connectionFor(node).isEmpty
|
||||
} {
|
||||
val connectionFactory = () ⇒ RemoteActorRef(remote.server, gossipingNode, remote.remoteDaemon.path, None)
|
||||
val connectionFactory = () ⇒ RemoteActorRef(remote.system.provider, remote.server, gossipingNode, remote.remoteDaemon.path, None)
|
||||
connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
|
||||
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes
|
||||
}
|
||||
|
|
@ -299,7 +299,7 @@ class Gossiper(remote: Remote) {
|
|||
}
|
||||
|
||||
private def toRemoteMessage(gossip: Gossip): RemoteProtocol.RemoteSystemDaemonMessageProtocol = {
|
||||
val gossipAsBytes = app.serialization.serialize(gossip) match {
|
||||
val gossipAsBytes = system.serialization.serialize(gossip) match {
|
||||
case Left(error) ⇒ throw error
|
||||
case Right(bytes) ⇒ bytes
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,15 +11,15 @@ import akka.actor.ActorSystem
|
|||
|
||||
object MessageSerializer {
|
||||
|
||||
def deserialize(app: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
|
||||
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
|
||||
val clazz = loadManifest(classLoader, messageProtocol)
|
||||
app.serialization.deserialize(messageProtocol.getMessage.toByteArray,
|
||||
system.serialization.deserialize(messageProtocol.getMessage.toByteArray,
|
||||
clazz, classLoader).fold(x ⇒ throw x, identity)
|
||||
}
|
||||
|
||||
def serialize(app: ActorSystem, message: AnyRef): MessageProtocol = {
|
||||
def serialize(system: ActorSystem, message: AnyRef): MessageProtocol = {
|
||||
val builder = MessageProtocol.newBuilder
|
||||
val bytes = app.serialization.serialize(message).fold(x ⇒ throw x, identity)
|
||||
val bytes = system.serialization.serialize(message).fold(x ⇒ throw x, identity)
|
||||
builder.setMessage(ByteString.copyFrom(bytes))
|
||||
builder.setMessageManifest(ByteString.copyFromUtf8(message.getClass.getName))
|
||||
builder.build
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.remote
|
|||
import scala.collection.mutable
|
||||
import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid }
|
||||
import akka.actor.Actor._
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.ActorSystemImpl
|
||||
|
||||
/**
|
||||
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
|
||||
|
|
@ -58,14 +58,14 @@ object NetworkEventStream {
|
|||
}
|
||||
}
|
||||
|
||||
class NetworkEventStream(val app: ActorSystem) {
|
||||
class NetworkEventStream(system: ActorSystemImpl) {
|
||||
|
||||
import NetworkEventStream._
|
||||
|
||||
// FIXME: check that this supervision is correct
|
||||
private[akka] val sender = app.provider.actorOf(
|
||||
Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
|
||||
app.systemGuardian, "network-event-sender", systemService = true)
|
||||
private[akka] val sender = system.provider.actorOf(system,
|
||||
Props[Channel].copy(dispatcher = system.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
|
||||
system.systemGuardian, "network-event-sender", systemService = true)
|
||||
|
||||
/**
|
||||
* Registers a network event stream listener (asyncronously).
|
||||
|
|
|
|||
|
|
@ -15,33 +15,30 @@ import akka.actor.DeploymentConfig._
|
|||
import akka.serialization.Compression.LZF
|
||||
import akka.remote.RemoteProtocol._
|
||||
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
import com.eaio.uuid.UUID
|
||||
import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression }
|
||||
import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
/**
|
||||
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Remote(val app: ActorSystem) {
|
||||
class Remote(val system: ActorSystemImpl, val nodename: String) {
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
import app._
|
||||
import app.config
|
||||
import app.AkkaConfig._
|
||||
import system._
|
||||
val AC = settings
|
||||
import AC._
|
||||
|
||||
val nodename = app.nodename
|
||||
|
||||
// TODO move to AkkaConfig?
|
||||
// TODO move to settings?
|
||||
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
|
||||
val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(app)
|
||||
val failureDetector = new AccrualFailureDetector(system)
|
||||
|
||||
// val gossiper = new Gossiper(this)
|
||||
|
||||
|
|
@ -50,17 +47,18 @@ class Remote(val app: ActorSystem) {
|
|||
// FIXME configure computeGridDispatcher to what?
|
||||
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
|
||||
|
||||
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
|
||||
// FIXME it is probably better to create another supervisor for handling the children created by handle_*
|
||||
private[remote] lazy val remoteDaemonSupervisor = system.actorOf(Props(
|
||||
OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want?
|
||||
|
||||
private[remote] lazy val remoteDaemon =
|
||||
app.provider.actorOf(
|
||||
system.provider.actorOf(system,
|
||||
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
|
||||
remoteDaemonSupervisor,
|
||||
remoteDaemonServiceName,
|
||||
systemService = true)
|
||||
|
||||
private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor {
|
||||
private[remote] lazy val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
|
|
@ -68,22 +66,22 @@ class Remote(val app: ActorSystem) {
|
|||
}
|
||||
}), "akka.remote.RemoteClientLifeCycleListener")
|
||||
|
||||
lazy val eventStream = new NetworkEventStream(app)
|
||||
lazy val eventStream = new NetworkEventStream(system)
|
||||
|
||||
lazy val server: RemoteSupport = {
|
||||
val remote = new akka.remote.netty.NettyRemoteSupport(app)
|
||||
val remote = new akka.remote.netty.NettyRemoteSupport(system)
|
||||
remote.start() //TODO FIXME Any application loader here?
|
||||
|
||||
app.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
app.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
// TODO actually register this provider in app in remote mode
|
||||
// TODO actually register this provider in system in remote mode
|
||||
//provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
|
||||
remote
|
||||
}
|
||||
|
||||
def start(): Unit = {
|
||||
val serverAddress = server.app.address //Force init of server
|
||||
val serverAddress = server.system.rootPath.remoteAddress //Force init of server
|
||||
val daemonAddress = remoteDaemon.address //Force init of daemon
|
||||
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress)
|
||||
}
|
||||
|
|
@ -99,6 +97,7 @@ class Remote(val app: ActorSystem) {
|
|||
class RemoteSystemDaemon(remote: Remote) extends Actor {
|
||||
|
||||
import remote._
|
||||
import remote.{ system ⇒ systemImpl }
|
||||
|
||||
override def preRestart(reason: Throwable, msg: Option[Any]) {
|
||||
log.debug("RemoteSystemDaemon failed due to [{}] - restarting...", reason)
|
||||
|
|
@ -135,16 +134,16 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
if (shouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
|
||||
|
||||
val actorFactory =
|
||||
app.serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match {
|
||||
system.serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match {
|
||||
case Left(error) ⇒ throw error
|
||||
case Right(instance) ⇒ instance.asInstanceOf[() ⇒ Actor]
|
||||
}
|
||||
|
||||
val actorPath = ActorPath(remote.app, message.getActorPath)
|
||||
val parent = actorPath.parent.ref
|
||||
val actorPath = ActorPath(systemImpl, message.getActorPath)
|
||||
val parent = system.actorFor(actorPath.parent)
|
||||
|
||||
if (parent.isDefined) {
|
||||
app.provider.actorOf(Props(creator = actorFactory), parent.get, actorPath.name)
|
||||
systemImpl.provider.actorOf(systemImpl, Props(creator = actorFactory), parent.get, actorPath.name)
|
||||
} else {
|
||||
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message)
|
||||
}
|
||||
|
|
@ -153,7 +152,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message)
|
||||
}
|
||||
|
||||
sender ! Success(app.address)
|
||||
sender ! Success(systemImpl.address)
|
||||
} catch {
|
||||
case error: Throwable ⇒ //FIXME doesn't seem sensible
|
||||
sender ! Failure(error)
|
||||
|
|
@ -182,40 +181,47 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
// }
|
||||
}
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
private val tempNumber = new AtomicLong
|
||||
def tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement())
|
||||
def tempPath = remoteDaemon.path / tempName
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
|
||||
new LocalActorRef(app,
|
||||
new LocalActorRef(systemImpl,
|
||||
Props(
|
||||
context ⇒ {
|
||||
case f: Function0[_] ⇒ try { f() } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
|
||||
}
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
|
||||
new LocalActorRef(app,
|
||||
new LocalActorRef(systemImpl,
|
||||
Props(
|
||||
context ⇒ {
|
||||
case f: Function0[_] ⇒ try { sender ! f() } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
|
||||
}
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
|
||||
new LocalActorRef(app,
|
||||
new LocalActorRef(systemImpl,
|
||||
Props(
|
||||
context ⇒ {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
||||
}
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
|
||||
new LocalActorRef(app,
|
||||
new LocalActorRef(systemImpl,
|
||||
Props(
|
||||
context ⇒ {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try { sender ! fun.asInstanceOf[Any ⇒ Any](param) } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
||||
}
|
||||
|
||||
def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
|
||||
|
|
@ -224,7 +230,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
}
|
||||
|
||||
private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = {
|
||||
app.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
|
||||
system.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
|
||||
case Left(error) ⇒ throw error
|
||||
case Right(instance) ⇒ instance.asInstanceOf[T]
|
||||
}
|
||||
|
|
@ -232,18 +238,21 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
}
|
||||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
|
||||
|
||||
val provider = remote.system.asInstanceOf[ActorSystemImpl].provider
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender)
|
||||
remote.app.provider.deserialize(
|
||||
provider.deserialize(
|
||||
SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES"))
|
||||
else
|
||||
remote.app.deadLetters
|
||||
remote.system.deadLetters
|
||||
|
||||
lazy val recipient: ActorRef = remote.app.actorFor(input.getRecipient.getPath).getOrElse(remote.app.deadLetters)
|
||||
lazy val recipient: ActorRef = remote.system.actorFor(input.getRecipient.getPath).getOrElse(remote.system.deadLetters)
|
||||
|
||||
lazy val payload: Either[Throwable, AnyRef] =
|
||||
if (input.hasException) Left(parseException())
|
||||
else Right(MessageSerializer.deserialize(remote.app, input.getMessage, classLoader))
|
||||
else Right(MessageSerializer.deserialize(remote.system, input.getMessage, classLoader))
|
||||
|
||||
protected def parseException(): Throwable = {
|
||||
val exception = input.getException
|
||||
|
|
@ -256,7 +265,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
|
|||
.newInstance(exception.getMessage).asInstanceOf[Throwable]
|
||||
} catch {
|
||||
case problem: Exception ⇒
|
||||
remote.app.eventStream.publish(Logging.Error(problem, remote, problem.getMessage))
|
||||
remote.system.eventStream.publish(Logging.Error(problem, remote, problem.getMessage))
|
||||
CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
|
||||
}
|
||||
}
|
||||
|
|
@ -266,7 +275,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo
|
|||
|
||||
trait RemoteMarshallingOps {
|
||||
|
||||
def app: ActorSystem
|
||||
def system: ActorSystem
|
||||
|
||||
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
|
|
@ -284,7 +293,7 @@ trait RemoteMarshallingOps {
|
|||
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
||||
*/
|
||||
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
||||
val rep = app.provider.serialize(actor)
|
||||
val rep = system.asInstanceOf[ActorSystemImpl].provider.serialize(actor)
|
||||
ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build
|
||||
}
|
||||
|
||||
|
|
@ -297,7 +306,7 @@ trait RemoteMarshallingOps {
|
|||
|
||||
message match {
|
||||
case Right(message) ⇒
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(app, message.asInstanceOf[AnyRef]))
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
case Left(exception) ⇒
|
||||
messageBuilder.setException(ExceptionProtocol.newBuilder
|
||||
.setClassname(exception.getClass.getName)
|
||||
|
|
|
|||
|
|
@ -17,47 +17,63 @@ import akka.serialization.{ Serialization, Serializer, Compression }
|
|||
import akka.serialization.Compression.LZF
|
||||
import akka.remote.RemoteProtocol._
|
||||
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import com.google.protobuf.ByteString
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import akka.event.EventStream
|
||||
|
||||
/**
|
||||
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
||||
class RemoteActorRefProvider(
|
||||
val settings: ActorSystem.Settings,
|
||||
val rootPath: ActorPath,
|
||||
val eventStream: EventStream,
|
||||
val dispatcher: MessageDispatcher,
|
||||
val scheduler: Scheduler) extends ActorRefProvider {
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(eventStream, this)
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.dispatch.Promise
|
||||
|
||||
val local = new LocalActorRefProvider(app)
|
||||
val remote = new Remote(app)
|
||||
val local = new LocalActorRefProvider(settings, rootPath, eventStream, dispatcher, scheduler)
|
||||
def deathWatch = local.deathWatch
|
||||
def guardian = local.guardian
|
||||
def systemGuardian = local.systemGuardian
|
||||
def nodename = local.nodename
|
||||
def tempName = local.tempName
|
||||
|
||||
@volatile
|
||||
var remote: Remote = _
|
||||
|
||||
private val actors = new ConcurrentHashMap[String, AnyRef]
|
||||
|
||||
private val remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
|
||||
@volatile
|
||||
private var remoteDaemonConnectionManager: RemoteConnectionManager = _
|
||||
|
||||
def init(system: ActorSystemImpl) {
|
||||
local.init(system)
|
||||
remote = new Remote(system, nodename)
|
||||
remoteDaemonConnectionManager = new RemoteConnectionManager(system, remote)
|
||||
}
|
||||
|
||||
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime
|
||||
private[akka] def terminationFuture = local.terminationFuture
|
||||
|
||||
private[akka] def deployer: Deployer = local.deployer
|
||||
|
||||
def defaultDispatcher = app.dispatcher
|
||||
def defaultTimeout = app.AkkaConfig.ActorTimeout
|
||||
def defaultDispatcher = dispatcher
|
||||
def defaultTimeout = settings.ActorTimeout
|
||||
|
||||
def scheduler: Scheduler = local.scheduler
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(system, props, supervisor, supervisor.path / name, systemService)
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(props, supervisor, supervisor.path / name, systemService)
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
|
||||
if (systemService) local.actorOf(props, supervisor, path, systemService)
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
|
||||
if (systemService) local.actorOf(system, props, supervisor, path, systemService)
|
||||
else {
|
||||
val name = path.name
|
||||
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
|
||||
|
|
@ -76,13 +92,13 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
// case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass)
|
||||
// }
|
||||
|
||||
def isReplicaNode: Boolean = remoteAddresses exists { _ == app.address }
|
||||
def isReplicaNode: Boolean = remoteAddresses exists { _ == system.address }
|
||||
|
||||
//app.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(app.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
||||
//system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
||||
|
||||
if (isReplicaNode) {
|
||||
// we are on one of the replica node for this remote actor
|
||||
local.actorOf(props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
|
||||
local.actorOf(system, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
|
||||
} else {
|
||||
|
||||
// we are on the single "reference" node uses the remote actors on the replica nodes
|
||||
|
|
@ -119,17 +135,17 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
|
||||
val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒
|
||||
val remoteAddress = RemoteAddress(a.hostname, a.port)
|
||||
conns + (remoteAddress -> RemoteActorRef(remote.server, remoteAddress, path, None))
|
||||
conns + (remoteAddress -> RemoteActorRef(remote.system.provider, remote.server, remoteAddress, path, None))
|
||||
}
|
||||
|
||||
val connectionManager = new RemoteConnectionManager(app, remote, connections)
|
||||
val connectionManager = new RemoteConnectionManager(system, remote, connections)
|
||||
|
||||
connections.keys foreach { useActorOnNode(_, path.toString, props.creator) }
|
||||
connections.keys foreach { useActorOnNode(system, _, path.toString, props.creator) }
|
||||
|
||||
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
|
||||
actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
|
||||
}
|
||||
|
||||
case deploy ⇒ local.actorOf(props, supervisor, name, systemService)
|
||||
case deploy ⇒ local.actorOf(system, props, supervisor, name, systemService)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
|
|
@ -137,7 +153,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
throw e
|
||||
}
|
||||
|
||||
// actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later
|
||||
// actor foreach system.registry.register // only for ActorRegistry backward compat, will be removed later
|
||||
|
||||
newFuture completeWithResult actor
|
||||
actors.replace(path.toString, newFuture, actor)
|
||||
|
|
@ -151,9 +167,9 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
* Copied from LocalActorRefProvider...
|
||||
*/
|
||||
// FIXME: implement supervision
|
||||
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
def actorOf(system: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
|
||||
new RoutedActorRef(app, props, supervisor, name)
|
||||
new RoutedActorRef(system, props, supervisor, name)
|
||||
}
|
||||
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = actors.get(ActorPath.join(path)) match {
|
||||
|
|
@ -162,6 +178,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
case future: Future[_] ⇒ Some(future.get.asInstanceOf[ActorRef])
|
||||
}
|
||||
|
||||
// TODO remove me
|
||||
val optimizeLocal = new AtomicBoolean(true)
|
||||
def optimizeLocalScoped_?() = optimizeLocal.get
|
||||
|
||||
|
|
@ -177,22 +194,22 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
|
||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
|
||||
val remoteAddress = RemoteAddress(actor.hostname, actor.port)
|
||||
if (optimizeLocalScoped_? && remoteAddress == app.address) {
|
||||
if (optimizeLocalScoped_? && remoteAddress == rootPath.remoteAddress) {
|
||||
local.actorFor(ActorPath.split(actor.path))
|
||||
} else {
|
||||
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress)
|
||||
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here
|
||||
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", rootPath.remoteAddress, actor.path, remoteAddress)
|
||||
Some(RemoteActorRef(remote.system.provider, remote.server, remoteAddress, rootPath / ActorPath.split(actor.path), None)) //Should it be None here
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on a specific node.
|
||||
*/
|
||||
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
||||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress)
|
||||
def useActorOnNode(system: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
||||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress)
|
||||
|
||||
val actorFactoryBytes =
|
||||
app.serialization.serialize(actorFactory) match {
|
||||
system.serialization.serialize(actorFactory) match {
|
||||
case Left(error) ⇒ throw error
|
||||
case Right(bytes) ⇒ if (remote.shouldCompressData) LZF.compress(bytes) else bytes
|
||||
}
|
||||
|
|
@ -242,8 +259,6 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
|
||||
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = local.ask(message, recipient, within)
|
||||
|
||||
private[akka] def dummyAskSender = local.dummyAskSender
|
||||
|
||||
private[akka] def tempPath = local.tempPath
|
||||
}
|
||||
|
||||
|
|
@ -254,6 +269,7 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
private[akka] case class RemoteActorRef private[akka] (
|
||||
provider: ActorRefProvider,
|
||||
remote: RemoteSupport,
|
||||
remoteAddress: RemoteAddress,
|
||||
path: ActorPath,
|
||||
|
|
@ -273,7 +289,7 @@ private[akka] case class RemoteActorRef private[akka] (
|
|||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), remoteAddress, this, loader)
|
||||
|
||||
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = remote.app.provider.ask(message, this, timeout)
|
||||
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = provider.ask(message, this, timeout)
|
||||
|
||||
def suspend(): Unit = ()
|
||||
|
||||
|
|
@ -289,7 +305,7 @@ private[akka] case class RemoteActorRef private[akka] (
|
|||
}
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def writeReplace(): AnyRef = remote.app.provider.serialize(this)
|
||||
private def writeReplace(): AnyRef = provider.serialize(this)
|
||||
|
||||
def startsWatching(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement
|
||||
|
||||
|
|
|
|||
|
|
@ -20,12 +20,12 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class RemoteConnectionManager(
|
||||
app: ActorSystem,
|
||||
system: ActorSystem,
|
||||
remote: Remote,
|
||||
initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef])
|
||||
extends ConnectionManager {
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(system, this)
|
||||
|
||||
// FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc.
|
||||
case class State(version: Long, connections: Map[RemoteAddress, ActorRef])
|
||||
|
|
@ -149,5 +149,5 @@ class RemoteConnectionManager(
|
|||
}
|
||||
|
||||
private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) =
|
||||
RemoteActorRef(remote.server, remoteAddress, actorPath, None)
|
||||
RemoteActorRef(remote.system.provider, remote.server, remoteAddress, actorPath, None)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ abstract class RemoteClient private[akka] (
|
|||
val remoteSupport: NettyRemoteSupport,
|
||||
val remoteAddress: RemoteAddress) {
|
||||
|
||||
val log = Logging(remoteSupport.app, this)
|
||||
val log = Logging(remoteSupport.system, this)
|
||||
|
||||
val name = simpleName(this) + "@" + remoteAddress
|
||||
|
||||
|
|
@ -148,7 +148,8 @@ class ActiveRemoteClient private[akka] (
|
|||
def sendSecureCookie(connection: ChannelFuture) {
|
||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(remoteSupport.app.address.hostname).setPort(remoteSupport.app.address.port).build)
|
||||
val addr = remoteSupport.system.rootPath.remoteAddress
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build)
|
||||
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
|
||||
}
|
||||
|
||||
|
|
@ -349,10 +350,10 @@ class ActiveRemoteClientHandler(
|
|||
/**
|
||||
* Provides the implementation of the Netty remote support
|
||||
*/
|
||||
class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
|
||||
class NettyRemoteSupport(_system: ActorSystem) extends RemoteSupport(_system) with RemoteMarshallingOps {
|
||||
|
||||
val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
|
||||
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
|
||||
val serverSettings = new RemoteServerSettings(system.settings.config, system.settings.DefaultTimeUnit)
|
||||
val clientSettings = new RemoteClientSettings(system.settings.config, system.settings.DefaultTimeUnit)
|
||||
|
||||
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
|
||||
private val clientsLock = new ReentrantReadWriteLock
|
||||
|
|
@ -447,7 +448,7 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
|
|||
|
||||
def name = currentServer.get match {
|
||||
case Some(server) ⇒ server.name
|
||||
case None ⇒ "Non-running NettyRemoteServer@" + app.address
|
||||
case None ⇒ "Non-running NettyRemoteServer@" + system.rootPath.remoteAddress
|
||||
}
|
||||
|
||||
private val _isRunning = new Switch(false)
|
||||
|
|
@ -479,9 +480,10 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
|
|||
}
|
||||
|
||||
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
|
||||
val log = Logging(remoteSupport.app, this)
|
||||
val log = Logging(remoteSupport.system, this)
|
||||
import remoteSupport.serverSettings._
|
||||
import remoteSupport.app.address
|
||||
|
||||
val address = remoteSupport.system.rootPath.remoteAddress
|
||||
|
||||
val name = "NettyRemoteServer@" + address
|
||||
|
||||
|
|
@ -583,7 +585,7 @@ class RemoteServerHandler(
|
|||
val applicationLoader: Option[ClassLoader],
|
||||
val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler {
|
||||
|
||||
val log = Logging(remoteSupport.app, this)
|
||||
val log = Logging(remoteSupport.system, this)
|
||||
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.testkit._
|
||||
import akka.actor.ActorSystemImpl
|
||||
|
||||
abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync {
|
||||
|
||||
|
|
@ -12,7 +13,7 @@ abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync {
|
|||
* Helper function for accessing the underlying remoting.
|
||||
*/
|
||||
def remote: Remote = {
|
||||
app.provider match {
|
||||
system.asInstanceOf[ActorSystemImpl].provider match {
|
||||
case r: RemoteActorRefProvider ⇒ r.remote
|
||||
case _ ⇒ throw new Exception("Remoting is not enabled")
|
||||
}
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue