clean up initialization of ActorSystem, fixes #1050
- create ActorSystemImpl trait to make ActorSystem fully abstract - add Java API for constructing (ActorSystem.create(...)) - only go through factory methods because .start() has become necessary - rename all user-facing occurrences of “app” to “system” (Actor trait and TestKit/AkkaSpec) - pass ActorSystemImpl to ActorRefs upon creation, which means that actorOf() and friends need such an argument, which must be provided to the ActorRefProvider by the ActorRefFactory implementation
This commit is contained in:
parent
6d85572ecc
commit
648661c548
83 changed files with 494 additions and 390 deletions
|
|
@ -9,7 +9,7 @@ import static org.junit.Assert.*;
|
|||
|
||||
public class JavaAPI {
|
||||
|
||||
private ActorSystem app = new ActorSystem();
|
||||
private ActorSystem app = ActorSystem.create();
|
||||
|
||||
@Test void mustBeAbleToCreateActorRefFromClass() {
|
||||
ActorRef ref = app.actorOf(JavaAPITestActor.class);
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import scala.Right;
|
|||
|
||||
public class JavaFutureTests {
|
||||
|
||||
private final ActorSystem app = new ActorSystem();
|
||||
private final ActorSystem app = ActorSystem.create();
|
||||
private final Timeout t = app.AkkaConfig().ActorTimeout();
|
||||
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);
|
||||
|
||||
|
|
|
|||
|
|
@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec {
|
|||
out.flush
|
||||
out.close
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
val readA = in.readObject
|
||||
|
||||
|
|
@ -284,14 +284,14 @@ class ActorRefSpec extends AkkaSpec {
|
|||
val baos = new ByteArrayOutputStream(8192 * 32)
|
||||
val out = new ObjectOutputStream(baos)
|
||||
|
||||
val serialized = SerializedActorRef(app.address.hostname, app.address.port, "/this/path/does/not/exist")
|
||||
val serialized = SerializedActorRef(system.root.remoteAddress.hostname, system.root.remoteAddress.port, "/this/path/does/not/exist")
|
||||
|
||||
out.writeObject(serialized)
|
||||
|
||||
out.flush
|
||||
out.close
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
(intercept[java.lang.IllegalStateException] {
|
||||
in.readObject
|
||||
|
|
|
|||
|
|
@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
|
|||
}
|
||||
}, timeout = t))
|
||||
|
||||
val defaultTimeout = app.AkkaConfig.ActorTimeout.duration
|
||||
val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
|
||||
val defaultTimeout = system.AkkaConfig.ActorTimeout.duration
|
||||
val testTimeout = if (system.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
|
||||
|
||||
"An Actor-based Future" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ class ClusterSpec extends AkkaSpec {
|
|||
"be able to parse 'akka.actor.cluster._' config elements" in {
|
||||
|
||||
// TODO: make it use its own special config?
|
||||
val config = app.config
|
||||
val config = system.AkkaConfig.config
|
||||
import config._
|
||||
|
||||
//akka.cluster
|
||||
|
|
|
|||
|
|
@ -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(
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
@ -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.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
|
||||
val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
|
||||
val baos = new ByteArrayOutputStream(8192 * 4)
|
||||
val out = new ObjectOutputStream(baos)
|
||||
|
||||
|
|
@ -341,7 +341,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
|
||||
|
||||
mNew.method must be(m.method)
|
||||
|
|
@ -351,7 +351,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
"be able to serialize and deserialize invocations' parameters" in {
|
||||
import java.io._
|
||||
val someFoo: Foo = new Bar
|
||||
val m = TypedActor.MethodCall(app.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
|
||||
val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
|
||||
val baos = new ByteArrayOutputStream(8192 * 4)
|
||||
val out = new ObjectOutputStream(baos)
|
||||
|
||||
|
|
@ -360,7 +360,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
|
||||
Serialization.app.withValue(app) {
|
||||
Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
|
||||
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
|
||||
|
||||
mNew.method must be(m.method)
|
||||
|
|
|
|||
|
|
@ -148,14 +148,14 @@ object ActorModelSpec {
|
|||
|
||||
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
|
||||
starts: Long = dispatcher.starts.get(),
|
||||
stops: Long = dispatcher.stops.get())(implicit app: ActorSystem) {
|
||||
stops: Long = dispatcher.stops.get())(implicit system: ActorSystem) {
|
||||
val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
|
||||
try {
|
||||
await(deadline)(starts == dispatcher.starts.get)
|
||||
await(deadline)(stops == dispatcher.stops.get)
|
||||
} catch {
|
||||
case e ⇒
|
||||
app.eventStream.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
|
||||
system.eventStream.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
|
||||
" required: starts=" + starts + ",stops=" + stops))
|
||||
throw e
|
||||
}
|
||||
|
|
@ -181,7 +181,7 @@ object ActorModelSpec {
|
|||
unregisters: Long = 0,
|
||||
msgsReceived: Long = 0,
|
||||
msgsProcessed: Long = 0,
|
||||
restarts: Long = 0)(implicit app: ActorSystem) {
|
||||
restarts: Long = 0)(implicit system: ActorSystem) {
|
||||
assertRef(actorRef, dispatcher)(
|
||||
suspensions,
|
||||
resumes,
|
||||
|
|
@ -199,7 +199,7 @@ object ActorModelSpec {
|
|||
unregisters: Long = statsFor(actorRef).unregisters.get(),
|
||||
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
|
||||
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
|
||||
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: ActorSystem) {
|
||||
restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
|
||||
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
|
||||
val deadline = System.currentTimeMillis + 1000
|
||||
try {
|
||||
|
|
@ -212,7 +212,7 @@ object ActorModelSpec {
|
|||
await(deadline)(stats.restarts.get() == restarts)
|
||||
} catch {
|
||||
case e ⇒
|
||||
app.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
|
||||
system.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
|
||||
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
|
||||
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
|
||||
throw e
|
||||
|
|
@ -235,7 +235,7 @@ abstract class ActorModelSpec extends AkkaSpec {
|
|||
|
||||
import ActorModelSpec._
|
||||
|
||||
def newTestActor(dispatcher: MessageDispatcher) = app.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||
def newTestActor(dispatcher: MessageDispatcher) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||
|
||||
protected def newInterceptedDispatcher: MessageDispatcherInterceptor
|
||||
protected def dispatcherType: String
|
||||
|
|
@ -318,7 +318,7 @@ abstract class ActorModelSpec extends AkkaSpec {
|
|||
try {
|
||||
f
|
||||
} catch {
|
||||
case e ⇒ app.eventStream.publish(Error(e, this, "error in spawned thread"))
|
||||
case e ⇒ system.eventStream.publish(Error(e, this, "error in spawned thread"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -421,10 +421,10 @@ class DispatcherModelSpec extends ActorModelSpec {
|
|||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", app.AkkaConfig.DispatcherThroughput,
|
||||
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
|
||||
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.AkkaConfig.DispatcherThroughput,
|
||||
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
|
||||
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
|
||||
def dispatcherType = "Dispatcher"
|
||||
|
||||
|
|
@ -458,14 +458,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
|
|||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", 1, // TODO check why 1 here? (came from old test)
|
||||
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
|
||||
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
new BalancingDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", 1, // TODO check why 1 here? (came from old test)
|
||||
system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
|
||||
config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
|
||||
def dispatcherType = "Balancing Dispatcher"
|
||||
|
||||
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = app.dispatcher
|
||||
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = system.dispatcher
|
||||
|
||||
"A " + dispatcherType must {
|
||||
"process messages in parallel" in {
|
||||
|
|
|
|||
|
|
@ -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.AkkaConfig.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
|
||||
|
|
|
|||
|
|
@ -773,7 +773,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
|
|||
|
||||
"ticket812FutureDispatchCleanup" in {
|
||||
filterException[FutureTimeoutException] {
|
||||
implicit val dispatcher = app.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build
|
||||
implicit val dispatcher = system.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build
|
||||
assert(dispatcher.tasks === 0)
|
||||
val future = Future({ Thread.sleep(100); "Done" }, 10)
|
||||
intercept[FutureTimeoutException] { future.await }
|
||||
|
|
|
|||
|
|
@ -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.AkkaConfig.MailboxPushTimeout))
|
||||
}
|
||||
}
|
||||
|
||||
def testOrdering(mboxType: MailboxType) {
|
||||
val dispatcher = app.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
|
||||
val dispatcher = system.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
|
||||
|
||||
val actor = actorOf(Props(new Actor {
|
||||
var acc: List[Int] = Nil
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.event
|
|||
import akka.testkit.AkkaSpec
|
||||
import akka.config.Configuration
|
||||
import akka.util.duration._
|
||||
import akka.actor.{ Actor, ActorRef }
|
||||
import akka.actor.{ Actor, ActorRef, ActorSystemImpl }
|
||||
|
||||
object EventStreamSpec {
|
||||
case class M(i: Int)
|
||||
|
|
@ -14,7 +14,7 @@ object EventStreamSpec {
|
|||
case class SetTarget(ref: ActorRef)
|
||||
|
||||
class MyLog extends Actor {
|
||||
var dst: ActorRef = app.deadLetters
|
||||
var dst: ActorRef = system.deadLetters
|
||||
def receive = {
|
||||
case Logging.InitializeLogger(bus) ⇒ bus.subscribe(context.self, classOf[SetTarget]); sender ! Logging.LoggerInitialized
|
||||
case SetTarget(ref) ⇒ dst = ref; dst ! "OK"
|
||||
|
|
@ -36,11 +36,13 @@ class EventStreamSpec extends AkkaSpec(Configuration(
|
|||
|
||||
import EventStreamSpec._
|
||||
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
|
||||
"An EventStream" must {
|
||||
|
||||
"manage subscriptions" in {
|
||||
val bus = new EventStream(true)
|
||||
bus.start(app.provider)
|
||||
bus.start(impl)
|
||||
bus.subscribe(testActor, classOf[M])
|
||||
bus.publish(M(42))
|
||||
within(1 second) {
|
||||
|
|
@ -53,8 +55,8 @@ class EventStreamSpec extends AkkaSpec(Configuration(
|
|||
|
||||
"manage log levels" in {
|
||||
val bus = new EventStream(false)
|
||||
bus.start(app.provider)
|
||||
bus.startDefaultLoggers(app.provider, app.AkkaConfig)
|
||||
bus.start(impl)
|
||||
bus.startDefaultLoggers(impl)
|
||||
bus.publish(SetTarget(testActor))
|
||||
expectMsg("OK")
|
||||
within(2 seconds) {
|
||||
|
|
@ -75,7 +77,7 @@ class EventStreamSpec extends AkkaSpec(Configuration(
|
|||
val b2 = new B2
|
||||
val c = new C
|
||||
val bus = new EventStream(false)
|
||||
bus.start(app.provider)
|
||||
bus.start(impl)
|
||||
within(2 seconds) {
|
||||
bus.subscribe(testActor, classOf[B2]) === true
|
||||
bus.publish(c)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -18,12 +18,12 @@ import akka.dispatch.Dispatchers
|
|||
class TellThroughputPerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughputPerformanceSpec._
|
||||
|
||||
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
||||
val destinationDispatcher = app.dispatcherFactory.newDispatcher("destination-dispatcher")
|
||||
val destinationDispatcher = system.dispatcherFactory.newDispatcher("destination-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
|
@ -71,9 +71,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
|
|||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
yield app.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
|
||||
val clients = for (dest ← destinations)
|
||||
yield app.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
|
||||
|
||||
val start = System.nanoTime
|
||||
clients.foreach(_ ! Run)
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -224,8 +224,8 @@ class Report(
|
|||
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
|
||||
sb.append("\n")
|
||||
sb.append("Akka config:")
|
||||
for (key ← app.config.keys) {
|
||||
sb.append("\n ").append(key).append("=").append(app.config(key))
|
||||
for (key ← app.AkkaConfig.config.keys) {
|
||||
sb.append("\n ").append(key).append("=").append(app.AkkaConfig.config(key))
|
||||
}
|
||||
|
||||
sb.toString
|
||||
|
|
|
|||
|
|
@ -329,7 +329,7 @@ class ActorPoolSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"support typed actors" in {
|
||||
val pool = app.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
|
||||
val pool = system.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
|
||||
def lowerBound = 1
|
||||
def upperBound = 5
|
||||
def pressureThreshold = 1
|
||||
|
|
@ -338,7 +338,7 @@ class ActorPoolSpec extends AkkaSpec {
|
|||
def rampupRate = 0.1
|
||||
def backoffRate = 0.50
|
||||
def backoffThreshold = 0.50
|
||||
def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
|
||||
def instance(p: Props) = system.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
|
||||
def receive = _route
|
||||
}, Props().withTimeout(10 seconds).withFaultHandler(faultHandler))
|
||||
|
||||
|
|
@ -348,7 +348,7 @@ class ActorPoolSpec extends AkkaSpec {
|
|||
val value = r.get
|
||||
value must equal(i * i)
|
||||
}
|
||||
app.typedActor.stop(pool)
|
||||
system.typedActor.stop(pool)
|
||||
}
|
||||
|
||||
"provide default supervision of pooled actors" in {
|
||||
|
|
|
|||
|
|
@ -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.app.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.deadLetterMailbox, app.eventStream, app.scheduler, "test") with MessageDispatcherInterceptor
|
||||
def newInterceptedDispatcher = new CallingThreadDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "test") with MessageDispatcherInterceptor
|
||||
def dispatcherType = "Calling Thread Dispatcher"
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -154,7 +154,7 @@ object Timeout {
|
|||
}
|
||||
|
||||
trait ActorLogging { this: Actor ⇒
|
||||
val log = akka.event.Logging(app.eventStream, context.self)
|
||||
val log = akka.event.Logging(system.eventStream, context.self)
|
||||
}
|
||||
|
||||
object Actor {
|
||||
|
|
@ -229,12 +229,12 @@ trait Actor {
|
|||
context
|
||||
}
|
||||
|
||||
implicit def app = context.app
|
||||
implicit def system = context.system
|
||||
|
||||
/**
|
||||
* The default timeout, based on the config setting 'akka.actor.timeout'
|
||||
*/
|
||||
implicit def defaultTimeout = app.AkkaConfig.ActorTimeout
|
||||
implicit def defaultTimeout = system.AkkaConfig.ActorTimeout
|
||||
|
||||
/**
|
||||
* Wrap a Receive partial function in a logging enclosure, which sends a
|
||||
|
|
@ -250,7 +250,7 @@ trait Actor {
|
|||
* This method does NOT modify the given Receive unless
|
||||
* akka.actor.debug.receive is set within akka.conf.
|
||||
*/
|
||||
def loggable(self: AnyRef)(r: Receive): Receive = if (app.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
|
||||
def loggable(self: AnyRef)(r: Receive): Receive = if (system.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
|
||||
|
||||
/**
|
||||
* Some[ActorRef] representation of the 'self' ActorRef reference.
|
||||
|
|
|
|||
|
|
@ -45,7 +45,7 @@ trait ActorContext extends ActorRefFactory with TypedActorFactory {
|
|||
|
||||
def handleChildTerminated(child: ActorRef): Unit
|
||||
|
||||
def app: ActorSystem
|
||||
def system: ActorSystem
|
||||
|
||||
def parent: ActorRef
|
||||
}
|
||||
|
|
@ -63,7 +63,7 @@ private[akka] object ActorCell {
|
|||
//vars don't need volatile since it's protected with the mailbox status
|
||||
//Make sure that they are not read/written outside of a message processing (systemInvoke/invoke)
|
||||
private[akka] class ActorCell(
|
||||
val app: ActorSystem,
|
||||
val app: ActorSystemImpl,
|
||||
val self: ActorRef with ScalaActorRef,
|
||||
val props: Props,
|
||||
val parent: ActorRef,
|
||||
|
|
@ -72,6 +72,8 @@ private[akka] class ActorCell(
|
|||
|
||||
import ActorCell._
|
||||
|
||||
final def system = app
|
||||
|
||||
protected final def guardian = self
|
||||
|
||||
protected def typedActor = app.typedActor
|
||||
|
|
|
|||
|
|
@ -4,6 +4,8 @@
|
|||
|
||||
package akka.actor
|
||||
|
||||
import akka.remote.RemoteAddress
|
||||
|
||||
object ActorPath {
|
||||
final val separator = "/"
|
||||
|
||||
|
|
@ -58,9 +60,9 @@ object ActorPath {
|
|||
*/
|
||||
trait ActorPath {
|
||||
/**
|
||||
* The akka application for this path.
|
||||
* The RemoteAddress for this path.
|
||||
*/
|
||||
def app: ActorSystem
|
||||
def remoteAddress: RemoteAddress
|
||||
|
||||
/**
|
||||
* The name of the actor that this path refers to.
|
||||
|
|
@ -78,9 +80,9 @@ trait ActorPath {
|
|||
def /(child: String): ActorPath
|
||||
|
||||
/**
|
||||
* Find the ActorRef for this path.
|
||||
* Recursively create a 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
|
||||
|
||||
|
|
|
|||
|
|
@ -13,6 +13,7 @@ import java.net.InetSocketAddress
|
|||
import akka.remote.RemoteAddress
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.event.EventStream
|
||||
import akka.event.DeathWatch
|
||||
|
||||
/**
|
||||
* ActorRef is an immutable and serializable handle to an Actor.
|
||||
|
|
@ -161,7 +162,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class LocalActorRef private[akka] (
|
||||
app: ActorSystem,
|
||||
app: ActorSystemImpl,
|
||||
_props: Props,
|
||||
_supervisor: ActorRef,
|
||||
val path: ActorPath,
|
||||
|
|
@ -172,7 +173,7 @@ class LocalActorRef private[akka] (
|
|||
|
||||
def name = path.name
|
||||
|
||||
def address: String = app.address + path.toString
|
||||
def address: String = path.toString
|
||||
|
||||
/*
|
||||
* actorCell.start() publishes actorCell & this to the dispatcher, which
|
||||
|
|
@ -415,16 +416,15 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) exte
|
|||
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
|
||||
}
|
||||
|
||||
abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef {
|
||||
abstract class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deathWatch: DeathWatch, timeout: Timeout, val dispatcher: MessageDispatcher) extends MinimalActorRef {
|
||||
final val result = new DefaultPromise[Any](timeout)(dispatcher)
|
||||
|
||||
// FIXME (actor path): put this under the tmp guardian supervisor
|
||||
val path: ActorPath = app.root / "tmp" / name
|
||||
override def name = path.name
|
||||
|
||||
def address: String = app.address + path.toString
|
||||
def address: String = path.toString
|
||||
|
||||
{
|
||||
val callback: Future[Any] ⇒ Unit = { _ ⇒ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
|
||||
val callback: Future[Any] ⇒ Unit = { _ ⇒ deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
|
||||
result onComplete callback
|
||||
result onTimeout callback
|
||||
}
|
||||
|
|
@ -450,5 +450,5 @@ abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = ap
|
|||
override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped"))
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def writeReplace(): AnyRef = app.provider.serialize(this)
|
||||
private def writeReplace(): AnyRef = provider.serialize(this)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import com.eaio.uuid.UUID
|
|||
*/
|
||||
trait ActorRefProvider {
|
||||
|
||||
def actorOf(props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(props, supervisor, name, false)
|
||||
def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(app, props, supervisor, name, false)
|
||||
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef]
|
||||
|
||||
|
|
@ -36,6 +36,8 @@ trait ActorRefProvider {
|
|||
|
||||
def AkkaConfig: ActorSystem.AkkaConfig
|
||||
|
||||
def init(app: ActorSystemImpl)
|
||||
|
||||
/**
|
||||
* What deployer will be used to resolve deployment configuration?
|
||||
*/
|
||||
|
|
@ -43,9 +45,9 @@ trait ActorRefProvider {
|
|||
|
||||
private[akka] def scheduler: Scheduler
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
|
||||
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
|
||||
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
|
||||
|
||||
private[akka] def evict(path: String): Boolean
|
||||
|
||||
|
|
@ -61,7 +63,7 @@ trait ActorRefProvider {
|
|||
|
||||
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
|
||||
|
||||
private[akka] def tempPath: String
|
||||
private[akka] def tempName: String
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -69,9 +71,11 @@ trait ActorRefProvider {
|
|||
*/
|
||||
trait ActorRefFactory {
|
||||
|
||||
def provider: ActorRefProvider
|
||||
protected def app: ActorSystemImpl
|
||||
|
||||
def dispatcher: MessageDispatcher
|
||||
protected def provider: ActorRefProvider
|
||||
|
||||
protected def dispatcher: MessageDispatcher
|
||||
|
||||
/**
|
||||
* Father of all children created by this interface.
|
||||
|
|
@ -85,7 +89,7 @@ trait ActorRefFactory {
|
|||
Helpers.base64(l)
|
||||
}
|
||||
|
||||
def actorOf(props: Props): ActorRef = provider.actorOf(props, guardian, randomName, false)
|
||||
def actorOf(props: Props): ActorRef = provider.actorOf(app, props, guardian, randomName, false)
|
||||
|
||||
/*
|
||||
* TODO this will have to go at some point, because creating two actors with
|
||||
|
|
@ -95,7 +99,7 @@ trait ActorRefFactory {
|
|||
def actorOf(props: Props, name: String): ActorRef = {
|
||||
if (name == null || name == "" || name.startsWith("$"))
|
||||
throw new ActorInitializationException("actor name must not be null, empty or start with $")
|
||||
provider.actorOf(props, guardian, name, false)
|
||||
provider.actorOf(app, props, guardian, name, false)
|
||||
}
|
||||
|
||||
def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]))
|
||||
|
|
@ -109,6 +113,8 @@ trait ActorRefFactory {
|
|||
|
||||
def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() ⇒ creator.create()))
|
||||
|
||||
def actorFor(path: ActorPath): Option[ActorRef] = actorFor(path.path)
|
||||
|
||||
def actorFor(path: String): Option[ActorRef] = actorFor(ActorPath.split(path))
|
||||
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = provider.actorFor(path)
|
||||
|
|
@ -120,7 +126,6 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
|
|||
* Local ActorRef provider.
|
||||
*/
|
||||
class LocalActorRefProvider(
|
||||
private val app: ActorSystem,
|
||||
val AkkaConfig: ActorSystem.AkkaConfig,
|
||||
val root: ActorPath,
|
||||
val eventStream: EventStream,
|
||||
|
|
@ -136,16 +141,18 @@ class LocalActorRefProvider(
|
|||
|
||||
private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename)
|
||||
|
||||
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher)
|
||||
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher)
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
private val tempNumber = new AtomicLong
|
||||
def tempPath = {
|
||||
def tempName = {
|
||||
val l = tempNumber.getAndIncrement()
|
||||
"$_" + Helpers.base64(l)
|
||||
}
|
||||
private val tempNode = root / "tmp"
|
||||
def tempPath = tempNode / tempName
|
||||
|
||||
// FIXME (actor path): this could become a cache for the new tree traversal actorFor
|
||||
// currently still used for tmp actors (e.g. ask actor refs)
|
||||
|
|
@ -159,12 +166,12 @@ class LocalActorRefProvider(
|
|||
@volatile
|
||||
var stopped = false
|
||||
|
||||
val name = app.name + "-bubble-walker"
|
||||
val name = "bubble-walker"
|
||||
|
||||
// FIXME (actor path): move the root path to the new root guardian
|
||||
val path = app.root
|
||||
val path = root / name
|
||||
|
||||
val address = app.address + path.toString
|
||||
val address = path.toString
|
||||
|
||||
override def toString = name
|
||||
|
||||
|
|
@ -208,20 +215,33 @@ class LocalActorRefProvider(
|
|||
}
|
||||
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
|
||||
|
||||
private val rootGuardian: ActorRef = actorOf(guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true)
|
||||
|
||||
val guardian: ActorRef = actorOf(guardianProps, rootGuardian, "app", true)
|
||||
|
||||
val systemGuardian: ActorRef = actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
|
||||
/*
|
||||
* The problem is that ActorRefs need a reference to the ActorSystem to
|
||||
* provide their service. Hence they cannot be created while the
|
||||
* constructors of ActorSystem and ActorRefProvider are still running.
|
||||
* The solution is to split out that last part into an init() method,
|
||||
* but it also requires these references to be @volatile.
|
||||
*/
|
||||
@volatile
|
||||
private var rootGuardian: ActorRef = _
|
||||
@volatile
|
||||
var guardian: ActorRef = _
|
||||
@volatile
|
||||
var systemGuardian: ActorRef = _
|
||||
|
||||
val deathWatch = createDeathWatch()
|
||||
|
||||
def init(app: ActorSystemImpl) {
|
||||
rootGuardian = actorOf(app, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true)
|
||||
guardian = actorOf(app, guardianProps, rootGuardian, "app", true)
|
||||
systemGuardian = actorOf(app, guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
|
||||
// chain death watchers so that killing guardian stops the application
|
||||
deathWatch.subscribe(systemGuardian, guardian)
|
||||
deathWatch.subscribe(rootGuardian, systemGuardian)
|
||||
}
|
||||
|
||||
// FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "app" name for now)
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(app.guardian), path.tail)
|
||||
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(guardian), path.tail)
|
||||
|
||||
@tailrec
|
||||
private def findInTree(start: Option[ActorRef], path: Iterable[String]): Option[ActorRef] = {
|
||||
|
|
@ -246,12 +266,12 @@ class LocalActorRefProvider(
|
|||
*/
|
||||
private[akka] def evict(path: String): Boolean = actors.remove(path) ne null
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(props, supervisor, supervisor.path / name, systemService)
|
||||
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(app, props, supervisor, supervisor.path / name, systemService)
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
|
||||
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
|
||||
val name = path.name
|
||||
val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout?
|
||||
val newFuture = Promise[ActorRef](5000)(dispatcher) // FIXME is this proper timeout?
|
||||
|
||||
actors.putIfAbsent(path.toString, newFuture) match {
|
||||
case null ⇒
|
||||
|
|
@ -270,7 +290,7 @@ class LocalActorRefProvider(
|
|||
case RouterType.Random ⇒ () ⇒ new RandomRouter
|
||||
case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter
|
||||
case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()(
|
||||
if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout)
|
||||
if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, AkkaConfig.ActorTimeout)
|
||||
case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
|
||||
case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
|
||||
case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet")
|
||||
|
|
@ -282,7 +302,7 @@ class LocalActorRefProvider(
|
|||
new LocalActorRef(app, props, supervisor, routedPath, systemService)
|
||||
}
|
||||
|
||||
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
|
||||
actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
|
||||
|
||||
case unknown ⇒ throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown)
|
||||
}
|
||||
|
|
@ -307,7 +327,7 @@ class LocalActorRefProvider(
|
|||
/**
|
||||
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
|
||||
*/
|
||||
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
// FIXME: this needs to take supervision into account!
|
||||
|
||||
//FIXME clustering should be implemented by cluster actor ref provider
|
||||
|
|
@ -324,16 +344,16 @@ class LocalActorRefProvider(
|
|||
}
|
||||
|
||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path))
|
||||
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(app.address, actor.path.toString)
|
||||
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(root.remoteAddress, actor.path.toString)
|
||||
|
||||
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
|
||||
|
||||
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = {
|
||||
import akka.dispatch.{ Future, Promise, DefaultPromise }
|
||||
(if (within == null) app.AkkaConfig.ActorTimeout else within) match {
|
||||
case t if t.duration.length <= 0 ⇒ new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout
|
||||
(if (within == null) AkkaConfig.ActorTimeout else within) match {
|
||||
case t if t.duration.length <= 0 ⇒ new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout
|
||||
case t ⇒
|
||||
val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this) }
|
||||
val a = new AskActorRef(tempPath, this, deathWatch, t, dispatcher) { def whenDone() = actors.remove(this) }
|
||||
assert(actors.putIfAbsent(a.path.toString, a) eq null) //If this fails, we're in deep trouble
|
||||
recipient.tell(message, a)
|
||||
a.result
|
||||
|
|
|
|||
|
|
@ -61,11 +61,14 @@ object ActorSystem {
|
|||
|
||||
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
|
||||
|
||||
def apply(name: String, config: Configuration) = new ActorSystem(name, config)
|
||||
def create(name: String, config: Configuration): ActorSystem = apply(name, config)
|
||||
def apply(name: String, config: Configuration): ActorSystem = new ActorSystemImpl(name, config).start()
|
||||
|
||||
def apply(name: String): ActorSystem = new ActorSystem(name)
|
||||
def create(name: String): ActorSystem = apply(name)
|
||||
def apply(name: String): ActorSystem = apply(name, defaultConfig)
|
||||
|
||||
def apply(): ActorSystem = new ActorSystem()
|
||||
def create(): ActorSystem = apply()
|
||||
def apply(): ActorSystem = apply("default")
|
||||
|
||||
sealed trait ExitStatus
|
||||
case object Stopped extends ExitStatus
|
||||
|
|
@ -126,22 +129,54 @@ object ActorSystem {
|
|||
|
||||
}
|
||||
|
||||
class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory {
|
||||
|
||||
abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
|
||||
import ActorSystem._
|
||||
|
||||
def this(name: String) = this(name, ActorSystem.defaultConfig)
|
||||
def this() = this("default")
|
||||
def name: String
|
||||
def AkkaConfig: AkkaConfig
|
||||
def nodename: String
|
||||
|
||||
val AkkaConfig = new AkkaConfig(config)
|
||||
|
||||
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true)
|
||||
|
||||
import AkkaConfig._
|
||||
/**
|
||||
* Construct a path below the application guardian.
|
||||
*/
|
||||
def /(name: String): ActorPath
|
||||
def root: ActorPath
|
||||
|
||||
val startTime = System.currentTimeMillis
|
||||
def uptime = (System.currentTimeMillis - startTime) / 1000
|
||||
|
||||
def eventStream: EventStream
|
||||
def log: LoggingAdapter
|
||||
|
||||
def deadLetters: ActorRef
|
||||
def deadLetterMailbox: Mailbox
|
||||
|
||||
// FIXME: Serialization should be an extension
|
||||
def serialization: Serialization
|
||||
// FIXME: TypedActor should be an extension
|
||||
def typedActor: TypedActor
|
||||
|
||||
def scheduler: Scheduler
|
||||
def dispatcherFactory: Dispatchers
|
||||
def dispatcher: MessageDispatcher
|
||||
|
||||
def registerOnTermination(code: ⇒ Unit)
|
||||
def registerOnTermination(code: Runnable)
|
||||
def stop()
|
||||
}
|
||||
|
||||
class ActorSystemImpl(val name: String, config: Configuration) extends ActorSystem {
|
||||
|
||||
import ActorSystem._
|
||||
|
||||
val AkkaConfig = new AkkaConfig(config)
|
||||
|
||||
protected def app = this
|
||||
|
||||
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true)
|
||||
|
||||
import AkkaConfig._
|
||||
|
||||
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
|
||||
case null | "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||
case value ⇒ value
|
||||
|
|
@ -158,7 +193,7 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
|
|||
/**
|
||||
* The root actor path for this application.
|
||||
*/
|
||||
val root: ActorPath = new RootActorPath(this)
|
||||
val root: ActorPath = new RootActorPath(address)
|
||||
|
||||
val deadLetters = new DeadLetterActorRef(eventStream, root / "nul")
|
||||
val deadLetterMailbox = new Mailbox(null) {
|
||||
|
|
@ -181,16 +216,21 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
|
|||
|
||||
deadLetters.init(dispatcher)
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val provider: ActorRefProvider = {
|
||||
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(b) ⇒ b
|
||||
}
|
||||
val params: Array[Class[_]] = Array(classOf[ActorSystem], classOf[AkkaConfig], classOf[ActorPath], classOf[EventStream], classOf[MessageDispatcher], classOf[Scheduler])
|
||||
val args: Array[AnyRef] = Array(this, AkkaConfig, root, eventStream, dispatcher, scheduler)
|
||||
val arguments = List(
|
||||
classOf[AkkaConfig] -> AkkaConfig,
|
||||
classOf[ActorPath] -> root,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[MessageDispatcher] -> dispatcher,
|
||||
classOf[Scheduler] -> scheduler)
|
||||
val types: Array[Class[_]] = arguments map (_._1) toArray
|
||||
val values: Array[AnyRef] = arguments map (_._2) toArray
|
||||
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, params, args) match {
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
|
|
@ -200,24 +240,33 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
|
|||
def guardian: ActorRef = provider.guardian
|
||||
def systemGuardian: ActorRef = provider.systemGuardian
|
||||
def deathWatch: DeathWatch = provider.deathWatch
|
||||
def nodename: String = provider.nodename
|
||||
|
||||
terminationFuture.onComplete(_ ⇒ scheduler.stop())
|
||||
terminationFuture.onComplete(_ ⇒ dispatcher.shutdown())
|
||||
|
||||
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
|
||||
eventStream.start(provider)
|
||||
eventStream.startDefaultLoggers(provider, AkkaConfig)
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
@volatile
|
||||
private var _typedActor: TypedActor = _
|
||||
def typedActor = _typedActor
|
||||
|
||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||
val serialization = new Serialization(this)
|
||||
|
||||
val typedActor = new TypedActor(AkkaConfig, serialization)
|
||||
|
||||
/**
|
||||
* Create an actor path under the application supervisor (/app).
|
||||
*/
|
||||
def /(actorName: String): ActorPath = guardian.path / actorName
|
||||
|
||||
def start(): this.type = {
|
||||
_serialization = new Serialization(this)
|
||||
_typedActor = new TypedActor(AkkaConfig, _serialization)
|
||||
provider.init(this)
|
||||
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
|
||||
eventStream.start(this)
|
||||
eventStream.startDefaultLoggers(this)
|
||||
this
|
||||
}
|
||||
|
||||
def registerOnTermination(code: ⇒ Unit) { terminationFuture onComplete (_ ⇒ code) }
|
||||
def registerOnTermination(code: Runnable) { terminationFuture onComplete (_ ⇒ code.run) }
|
||||
|
||||
// TODO shutdown all that other stuff, whatever that may be
|
||||
def stop() {
|
||||
guardian.stop()
|
||||
|
|
|
|||
|
|
@ -188,7 +188,7 @@ trait FSM[S, D] extends ListenerManagement {
|
|||
type Timeout = Option[Duration]
|
||||
type TransitionHandler = PartialFunction[(S, S), Unit]
|
||||
|
||||
val log = Logging(app.eventStream, context.self)
|
||||
val log = Logging(system, context.self)
|
||||
|
||||
/**
|
||||
* ****************************************
|
||||
|
|
@ -522,7 +522,7 @@ trait FSM[S, D] extends ListenerManagement {
|
|||
if (timeout.isDefined) {
|
||||
val t = timeout.get
|
||||
if (t.finite_? && t.length >= 0) {
|
||||
timeoutFuture = Some(app.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
|
||||
timeoutFuture = Some(system.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -565,7 +565,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
|
|||
|
||||
def logDepth: Int = 0
|
||||
|
||||
private val debugEvent = context.app.AkkaConfig.FsmDebugEvent
|
||||
private val debugEvent = system.AkkaConfig.FsmDebugEvent
|
||||
|
||||
private val events = new Array[Event](logDepth)
|
||||
private val states = new Array[AnyRef](logDepth)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -330,7 +330,7 @@ class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization)
|
|||
def receive = {
|
||||
case m: MethodCall ⇒
|
||||
TypedActor.selfReference set proxyVar.get
|
||||
TypedActor.appReference set app
|
||||
TypedActor.appReference set system
|
||||
try {
|
||||
if (m.isOneWay) m(me)
|
||||
else {
|
||||
|
|
|
|||
|
|
@ -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.provider.nodename)
|
||||
def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename)
|
||||
|
||||
def unapply(other: Any) = other match {
|
||||
case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName))
|
||||
|
|
|
|||
|
|
@ -3,11 +3,8 @@
|
|||
*/
|
||||
package akka.event
|
||||
|
||||
import akka.actor.{ ActorRef, Actor, Props }
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.Terminated
|
||||
import akka.actor.{ ActorRef, Actor, Props, ActorSystemImpl, Terminated }
|
||||
import akka.util.Subclassification
|
||||
import akka.actor.ActorRefProvider
|
||||
|
||||
class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification {
|
||||
|
||||
|
|
@ -42,13 +39,13 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas
|
|||
super.unsubscribe(subscriber)
|
||||
}
|
||||
|
||||
def start(provider: ActorRefProvider) {
|
||||
reaper = provider.actorOf(Props(new Actor {
|
||||
def start(app: ActorSystemImpl) {
|
||||
reaper = app.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case ref: ActorRef ⇒ watch(ref)
|
||||
case Terminated(ref) ⇒ unsubscribe(ref)
|
||||
}
|
||||
}), provider.systemGuardian, "MainBusReaper", true)
|
||||
}), "MainBusReaper")
|
||||
subscribers foreach (reaper ! _)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.event
|
||||
|
||||
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, simpleName }
|
||||
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, ActorSystemImpl, simpleName }
|
||||
import akka.AkkaException
|
||||
import akka.actor.ActorSystem.AkkaConfig
|
||||
import akka.util.ReflectiveAccess
|
||||
|
|
@ -79,13 +79,13 @@ trait LoggingBus extends ActorEventBus {
|
|||
publish(Info(this, "StandardOutLogger started"))
|
||||
}
|
||||
|
||||
private[akka] def startDefaultLoggers(provider: ActorRefProvider, config: AkkaConfig) {
|
||||
val level = levelFor(config.LogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel))
|
||||
private[akka] def startDefaultLoggers(app: ActorSystemImpl) {
|
||||
val level = levelFor(app.AkkaConfig.LogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.AkkaConfig.LogLevel))
|
||||
ErrorLevel
|
||||
}
|
||||
try {
|
||||
val defaultLoggers = config.EventHandlers match {
|
||||
val defaultLoggers = app.AkkaConfig.EventHandlers match {
|
||||
case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil
|
||||
case loggers ⇒ loggers
|
||||
}
|
||||
|
|
@ -95,7 +95,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
} yield {
|
||||
try {
|
||||
ReflectiveAccess.getClassFor[Actor](loggerName) match {
|
||||
case Right(actorClass) ⇒ addLogger(provider, actorClass, level)
|
||||
case Right(actorClass) ⇒ addLogger(app, actorClass, level)
|
||||
case Left(exception) ⇒ throw exception
|
||||
}
|
||||
} catch {
|
||||
|
|
@ -138,9 +138,9 @@ trait LoggingBus extends ActorEventBus {
|
|||
publish(Info(this, "all default loggers stopped"))
|
||||
}
|
||||
|
||||
private def addLogger(provider: ActorRefProvider, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
|
||||
private def addLogger(app: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
|
||||
val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz)
|
||||
val actor = provider.actorOf(Props(clazz), provider.systemGuardian, name, true)
|
||||
val actor = app.systemActorOf(Props(clazz), name)
|
||||
implicit val timeout = Timeout(3 seconds)
|
||||
val response = try actor ? InitializeLogger(this) get catch {
|
||||
case _: FutureTimeoutException ⇒
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.serialization
|
|||
|
||||
import akka.AkkaException
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import scala.util.DynamicVariable
|
||||
import akka.remote.RemoteSupport
|
||||
|
||||
|
|
@ -16,7 +16,7 @@ case class NoSerializerFoundException(m: String) extends AkkaException(m)
|
|||
* Serialization module. Contains methods for serialization and deserialization as well as
|
||||
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
|
||||
*/
|
||||
class Serialization(val app: ActorSystem) {
|
||||
class Serialization(val app: ActorSystemImpl) {
|
||||
|
||||
//TODO document me
|
||||
def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
|
||||
|
|
@ -70,7 +70,7 @@ class Serialization(val app: ActorSystem) {
|
|||
* But "default" can be overridden in config
|
||||
*/
|
||||
val serializers: Map[String, Serializer] =
|
||||
app.config.getSection("akka.actor.serializers")
|
||||
app.AkkaConfig.config.getSection("akka.actor.serializers")
|
||||
.map(_.map)
|
||||
.getOrElse(Map())
|
||||
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) {
|
||||
|
|
@ -81,7 +81,7 @@ class Serialization(val app: ActorSystem) {
|
|||
/**
|
||||
* bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used
|
||||
*/
|
||||
val bindings: Map[String, String] = app.config.getSection("akka.actor.serialization-bindings") map {
|
||||
val bindings: Map[String, String] = app.AkkaConfig.config.getSection("akka.actor.serialization-bindings") map {
|
||||
_.map.foldLeft(Map[String, String]()) {
|
||||
case (result, (k: String, vs: List[_])) ⇒ result ++ (vs collect { case v: String ⇒ (v, k) }) //All keys which are lists, take the Strings from them and Map them
|
||||
case (result, _) ⇒ result //For any other values, just skip them, TODO: print out warnings?
|
||||
|
|
@ -102,6 +102,6 @@ class Serialization(val app: ActorSystem) {
|
|||
|
||||
object Serialization {
|
||||
// TODO ensure that these are always set (i.e. withValue()) when doing deserialization
|
||||
val app = new DynamicVariable[ActorSystem](null)
|
||||
val app = new DynamicVariable[ActorSystemImpl](null)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -27,8 +27,8 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
|
||||
def this(app: ActorSystem) {
|
||||
this(
|
||||
app.config.getInt("akka.remote.failure-detector.theshold", 8),
|
||||
app.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
|
||||
app.AkkaConfig.config.getInt("akka.remote.failure-detector.theshold", 8),
|
||||
app.AkkaConfig.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
|
||||
}
|
||||
|
||||
private final val PhiFactor = 1.0 / math.log(10.0)
|
||||
|
|
|
|||
|
|
@ -107,7 +107,7 @@ class Gossiper(remote: Remote) {
|
|||
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
|
||||
private val seeds = Set(address) // FIXME read in list of seeds from config
|
||||
|
||||
private val address = app.address
|
||||
private val address = app.root.remoteAddress
|
||||
private val nodeFingerprint = address.##
|
||||
|
||||
private val random = SecureRandom.getInstance("SHA1PRNG")
|
||||
|
|
@ -153,7 +153,7 @@ class Gossiper(remote: Remote) {
|
|||
node ← oldAvailableNodes
|
||||
if connectionManager.connectionFor(node).isEmpty
|
||||
} {
|
||||
val connectionFactory = () ⇒ RemoteActorRef(remote.server, gossipingNode, remote.remoteDaemon.path, None)
|
||||
val connectionFactory = () ⇒ RemoteActorRef(remote.app.provider, remote.server, gossipingNode, remote.remoteDaemon.path, None)
|
||||
connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
|
||||
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.remote
|
|||
import scala.collection.mutable
|
||||
import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid }
|
||||
import akka.actor.Actor._
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.ActorSystemImpl
|
||||
|
||||
/**
|
||||
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
|
||||
|
|
@ -58,12 +58,12 @@ object NetworkEventStream {
|
|||
}
|
||||
}
|
||||
|
||||
class NetworkEventStream(val app: ActorSystem) {
|
||||
class NetworkEventStream(app: ActorSystemImpl) {
|
||||
|
||||
import NetworkEventStream._
|
||||
|
||||
// FIXME: check that this supervision is correct
|
||||
private[akka] val sender = app.provider.actorOf(
|
||||
private[akka] val sender = app.provider.actorOf(app,
|
||||
Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
|
||||
app.systemGuardian, "network-event-sender", systemService = true)
|
||||
|
||||
|
|
|
|||
|
|
@ -15,25 +15,24 @@ import akka.actor.DeploymentConfig._
|
|||
import akka.serialization.Compression.LZF
|
||||
import akka.remote.RemoteProtocol._
|
||||
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
import com.eaio.uuid.UUID
|
||||
import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression }
|
||||
import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
/**
|
||||
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Remote(val app: ActorSystem, val nodename: String) {
|
||||
class Remote(val app: ActorSystemImpl, val nodename: String) {
|
||||
|
||||
val log = Logging(app, this)
|
||||
|
||||
import app._
|
||||
import app.config
|
||||
import app.AkkaConfig._
|
||||
val AC = AkkaConfig
|
||||
import AC._
|
||||
|
||||
// TODO move to AkkaConfig?
|
||||
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
|
||||
|
|
@ -48,11 +47,12 @@ class Remote(val app: ActorSystem, val nodename: String) {
|
|||
// FIXME configure computeGridDispatcher to what?
|
||||
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
|
||||
|
||||
// FIXME it is probably better to create another supervisor for handling the children created by handle_*
|
||||
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
|
||||
OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want?
|
||||
|
||||
private[remote] lazy val remoteDaemon =
|
||||
app.provider.actorOf(
|
||||
app.provider.actorOf(app,
|
||||
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
|
||||
remoteDaemonSupervisor,
|
||||
remoteDaemonServiceName,
|
||||
|
|
@ -81,7 +81,7 @@ class Remote(val app: ActorSystem, val nodename: String) {
|
|||
}
|
||||
|
||||
def start(): Unit = {
|
||||
val serverAddress = server.app.address //Force init of server
|
||||
val serverAddress = server.app.root.remoteAddress //Force init of server
|
||||
val daemonAddress = remoteDaemon.address //Force init of daemon
|
||||
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress)
|
||||
}
|
||||
|
|
@ -139,10 +139,10 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
}
|
||||
|
||||
val actorPath = ActorPath(remote.app, message.getActorPath)
|
||||
val parent = actorPath.parent.ref
|
||||
val parent = app.actorFor(actorPath.parent)
|
||||
|
||||
if (parent.isDefined) {
|
||||
app.provider.actorOf(Props(creator = actorFactory), parent.get, actorPath.name)
|
||||
app.provider.actorOf(app, Props(creator = actorFactory), parent.get, actorPath.name)
|
||||
} else {
|
||||
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message)
|
||||
}
|
||||
|
|
@ -180,13 +180,23 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
// }
|
||||
}
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
private val tempNumber = new AtomicLong
|
||||
def tempName = {
|
||||
val l = tempNumber.getAndIncrement()
|
||||
"$_" + Helpers.base64(l)
|
||||
}
|
||||
def tempPath = remoteDaemon.path / tempName
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
|
||||
new LocalActorRef(app,
|
||||
Props(
|
||||
context ⇒ {
|
||||
case f: Function0[_] ⇒ try { f() } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
|
||||
}
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
|
|
@ -195,7 +205,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
Props(
|
||||
context ⇒ {
|
||||
case f: Function0[_] ⇒ try { sender ! f() } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
|
||||
}
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
|
|
@ -204,7 +214,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
Props(
|
||||
context ⇒ {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
||||
}
|
||||
|
||||
// FIXME: handle real remote supervision
|
||||
|
|
@ -213,7 +223,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
Props(
|
||||
context ⇒ {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try { sender ! fun.asInstanceOf[Any ⇒ Any](param) } finally { context.self.stop() }
|
||||
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
||||
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
||||
}
|
||||
|
||||
def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
|
||||
|
|
@ -230,9 +240,12 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
}
|
||||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
|
||||
|
||||
val provider = remote.app.asInstanceOf[ActorSystemImpl].provider
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender)
|
||||
remote.app.provider.deserialize(
|
||||
provider.deserialize(
|
||||
SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES"))
|
||||
else
|
||||
remote.app.deadLetters
|
||||
|
|
@ -282,7 +295,7 @@ trait RemoteMarshallingOps {
|
|||
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
||||
*/
|
||||
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
||||
val rep = app.provider.serialize(actor)
|
||||
val rep = app.asInstanceOf[ActorSystemImpl].provider.serialize(actor)
|
||||
ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -29,43 +29,51 @@ import akka.event.EventStream
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class RemoteActorRefProvider(
|
||||
val app: ActorSystem,
|
||||
val AkkaConfig: ActorSystem.AkkaConfig,
|
||||
val root: ActorPath,
|
||||
val eventStream: EventStream,
|
||||
val dispatcher: MessageDispatcher,
|
||||
val scheduler: Scheduler) extends ActorRefProvider {
|
||||
|
||||
val log = Logging(app, this)
|
||||
val log = Logging(eventStream, this)
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.dispatch.Promise
|
||||
|
||||
val local = new LocalActorRefProvider(app, AkkaConfig, root, eventStream, dispatcher, scheduler)
|
||||
val local = new LocalActorRefProvider(AkkaConfig, root, eventStream, dispatcher, scheduler)
|
||||
def deathWatch = local.deathWatch
|
||||
def guardian = local.guardian
|
||||
def systemGuardian = local.systemGuardian
|
||||
def nodename = local.nodename
|
||||
def tempName = local.tempName
|
||||
|
||||
val remote = new Remote(app, nodename)
|
||||
@volatile
|
||||
var remote: Remote = _
|
||||
|
||||
private val actors = new ConcurrentHashMap[String, AnyRef]
|
||||
|
||||
private val remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
|
||||
@volatile
|
||||
private var remoteDaemonConnectionManager: RemoteConnectionManager = _
|
||||
|
||||
def init(app: ActorSystemImpl) {
|
||||
local.init(app)
|
||||
remote = new Remote(app, nodename)
|
||||
remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
|
||||
}
|
||||
|
||||
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime
|
||||
private[akka] def terminationFuture = local.terminationFuture
|
||||
|
||||
private[akka] def deployer: Deployer = local.deployer
|
||||
|
||||
def defaultDispatcher = app.dispatcher
|
||||
def defaultTimeout = app.AkkaConfig.ActorTimeout
|
||||
def defaultDispatcher = dispatcher
|
||||
def defaultTimeout = AkkaConfig.ActorTimeout
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(props, supervisor, supervisor.path / name, systemService)
|
||||
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
actorOf(app, props, supervisor, supervisor.path / name, systemService)
|
||||
|
||||
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
|
||||
if (systemService) local.actorOf(props, supervisor, path, systemService)
|
||||
private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
|
||||
if (systemService) local.actorOf(app, props, supervisor, path, systemService)
|
||||
else {
|
||||
val name = path.name
|
||||
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
|
||||
|
|
@ -90,7 +98,7 @@ class RemoteActorRefProvider(
|
|||
|
||||
if (isReplicaNode) {
|
||||
// we are on one of the replica node for this remote actor
|
||||
local.actorOf(props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
|
||||
local.actorOf(app, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
|
||||
} else {
|
||||
|
||||
// we are on the single "reference" node uses the remote actors on the replica nodes
|
||||
|
|
@ -127,17 +135,17 @@ class RemoteActorRefProvider(
|
|||
|
||||
val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒
|
||||
val remoteAddress = RemoteAddress(a.hostname, a.port)
|
||||
conns + (remoteAddress -> RemoteActorRef(remote.server, remoteAddress, path, None))
|
||||
conns + (remoteAddress -> RemoteActorRef(remote.app.provider, remote.server, remoteAddress, path, None))
|
||||
}
|
||||
|
||||
val connectionManager = new RemoteConnectionManager(app, remote, connections)
|
||||
|
||||
connections.keys foreach { useActorOnNode(_, path.toString, props.creator) }
|
||||
connections.keys foreach { useActorOnNode(app, _, path.toString, props.creator) }
|
||||
|
||||
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
|
||||
actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
|
||||
}
|
||||
|
||||
case deploy ⇒ local.actorOf(props, supervisor, name, systemService)
|
||||
case deploy ⇒ local.actorOf(app, props, supervisor, name, systemService)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
|
|
@ -159,7 +167,7 @@ class RemoteActorRefProvider(
|
|||
* Copied from LocalActorRefProvider...
|
||||
*/
|
||||
// FIXME: implement supervision
|
||||
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
|
||||
if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
|
||||
new RoutedActorRef(app, props, supervisor, name)
|
||||
}
|
||||
|
|
@ -170,6 +178,7 @@ class RemoteActorRefProvider(
|
|||
case future: Future[_] ⇒ Some(future.get.asInstanceOf[ActorRef])
|
||||
}
|
||||
|
||||
// TODO remove me
|
||||
val optimizeLocal = new AtomicBoolean(true)
|
||||
def optimizeLocalScoped_?() = optimizeLocal.get
|
||||
|
||||
|
|
@ -185,19 +194,19 @@ class RemoteActorRefProvider(
|
|||
|
||||
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
|
||||
val remoteAddress = RemoteAddress(actor.hostname, actor.port)
|
||||
if (optimizeLocalScoped_? && remoteAddress == app.address) {
|
||||
if (optimizeLocalScoped_? && remoteAddress == root.remoteAddress) {
|
||||
local.actorFor(ActorPath.split(actor.path))
|
||||
} else {
|
||||
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress)
|
||||
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here
|
||||
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", root.remoteAddress, actor.path, remoteAddress)
|
||||
Some(RemoteActorRef(remote.app.provider, remote.server, remoteAddress, root / ActorPath.split(actor.path), None)) //Should it be None here
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on a specific node.
|
||||
*/
|
||||
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
||||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress)
|
||||
def useActorOnNode(app: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
||||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", root, actorPath, remoteAddress)
|
||||
|
||||
val actorFactoryBytes =
|
||||
app.serialization.serialize(actorFactory) match {
|
||||
|
|
@ -260,6 +269,7 @@ class RemoteActorRefProvider(
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
private[akka] case class RemoteActorRef private[akka] (
|
||||
provider: ActorRefProvider,
|
||||
remote: RemoteSupport,
|
||||
remoteAddress: RemoteAddress,
|
||||
path: ActorPath,
|
||||
|
|
@ -279,7 +289,7 @@ private[akka] case class RemoteActorRef private[akka] (
|
|||
|
||||
def tell(message: Any, sender: ActorRef): Unit = remote.send(message, Option(sender), remoteAddress, this, loader)
|
||||
|
||||
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = remote.app.provider.ask(message, this, timeout)
|
||||
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = provider.ask(message, this, timeout)
|
||||
|
||||
def suspend(): Unit = ()
|
||||
|
||||
|
|
@ -295,7 +305,7 @@ private[akka] case class RemoteActorRef private[akka] (
|
|||
}
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def writeReplace(): AnyRef = remote.app.provider.serialize(this)
|
||||
private def writeReplace(): AnyRef = provider.serialize(this)
|
||||
|
||||
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement
|
||||
|
||||
|
|
|
|||
|
|
@ -149,5 +149,5 @@ class RemoteConnectionManager(
|
|||
}
|
||||
|
||||
private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) =
|
||||
RemoteActorRef(remote.server, remoteAddress, actorPath, None)
|
||||
RemoteActorRef(remote.app.provider, remote.server, remoteAddress, actorPath, None)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -147,7 +147,8 @@ class ActiveRemoteClient private[akka] (
|
|||
def sendSecureCookie(connection: ChannelFuture) {
|
||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(remoteSupport.app.address.hostname).setPort(remoteSupport.app.address.port).build)
|
||||
val addr = remoteSupport.app.root.remoteAddress
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build)
|
||||
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
|
||||
}
|
||||
|
||||
|
|
@ -350,8 +351,8 @@ class ActiveRemoteClientHandler(
|
|||
*/
|
||||
class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
|
||||
|
||||
val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
|
||||
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit)
|
||||
val serverSettings = new RemoteServerSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
|
||||
val clientSettings = new RemoteClientSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
|
||||
|
||||
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
|
||||
private val clientsLock = new ReadWriteGuard
|
||||
|
|
@ -428,7 +429,7 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
|
|||
|
||||
def name = currentServer.get match {
|
||||
case Some(server) ⇒ server.name
|
||||
case None ⇒ "Non-running NettyRemoteServer@" + app.address
|
||||
case None ⇒ "Non-running NettyRemoteServer@" + app.root.remoteAddress
|
||||
}
|
||||
|
||||
private val _isRunning = new Switch(false)
|
||||
|
|
@ -459,7 +460,8 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
|
|||
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
|
||||
val log = Logging(remoteSupport.app, this)
|
||||
import remoteSupport.serverSettings._
|
||||
import remoteSupport.app.address
|
||||
|
||||
val address = remoteSupport.app.root.remoteAddress
|
||||
|
||||
val name = "NettyRemoteServer@" + address
|
||||
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec {
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "identify" ⇒ sender ! app.provider.nodename
|
||||
case "identify" ⇒ sender ! system.nodename
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -47,7 +47,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec {
|
|||
|
||||
barrier("start")
|
||||
|
||||
val actor = app.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val result = (actor ? "identify").get
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ object NewRemoteActorMultiJvmSpec {
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "identify" ⇒ sender ! app.provider.nodename
|
||||
case "identify" ⇒ sender ! system.nodename
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -46,7 +46,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec {
|
|||
|
||||
barrier("start")
|
||||
|
||||
val actor = app.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
val result = (actor ? "identify").get
|
||||
result must equal("node1")
|
||||
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ object RandomRoutedRemoteActorMultiJvmSpec {
|
|||
val NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! app.provider.nodename
|
||||
case "hit" ⇒ sender ! system.nodename
|
||||
case "end" ⇒ self.stop()
|
||||
}
|
||||
}
|
||||
|
|
@ -70,7 +70,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
|
|||
remote.start()
|
||||
|
||||
barrier("start")
|
||||
val actor = app.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = NrOfNodes - 1
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec {
|
|||
val NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! app.provider.nodename
|
||||
case "hit" ⇒ sender ! system.nodename
|
||||
case "end" ⇒ self.stop()
|
||||
}
|
||||
}
|
||||
|
|
@ -70,7 +70,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
|
|||
remote.start()
|
||||
|
||||
barrier("start")
|
||||
val actor = app.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = NrOfNodes - 1
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec {
|
|||
val NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! app.provider.nodename
|
||||
case "hit" ⇒ sender ! system.nodename
|
||||
case "end" ⇒ self.stop()
|
||||
}
|
||||
}
|
||||
|
|
@ -70,7 +70,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
|
|||
remote.start()
|
||||
|
||||
barrier("start")
|
||||
val actor = app.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true)
|
||||
|
||||
|
|
|
|||
|
|
@ -77,7 +77,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
|
|||
case Taken(`chopstickToWaitFor`) ⇒
|
||||
println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address)
|
||||
become(eating)
|
||||
app.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS)
|
||||
system.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS)
|
||||
|
||||
case Busy(chopstick) ⇒
|
||||
become(thinking)
|
||||
|
|
@ -106,7 +106,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
|
|||
left ! Put(self)
|
||||
right ! Put(self)
|
||||
println("%s puts down his chopsticks and starts to think", name)
|
||||
app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
|
||||
system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
|
||||
}
|
||||
|
||||
//All hakkers start in a non-eating state
|
||||
|
|
@ -114,7 +114,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
|
|||
case Think ⇒
|
||||
println("%s starts to think", name)
|
||||
become(thinking)
|
||||
app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
|
||||
system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ case class TakenBy(hakker: ActorRef)
|
|||
class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] {
|
||||
|
||||
// A chopstick begins its existence as available and taken by no one
|
||||
startWith(Available, TakenBy(app.deadLetters))
|
||||
startWith(Available, TakenBy(system.deadLetters))
|
||||
|
||||
// When a chopstick is available, it can be taken by a some hakker
|
||||
when(Available) {
|
||||
|
|
@ -47,7 +47,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] {
|
|||
case Event(Take, currentState) ⇒
|
||||
stay replying Busy(self)
|
||||
case Event(Put, TakenBy(hakker)) if sender == hakker ⇒
|
||||
goto(Available) using TakenBy(app.deadLetters)
|
||||
goto(Available) using TakenBy(system.deadLetters)
|
||||
}
|
||||
|
||||
// Initialze the chopstick
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ public class EitherOrElseExample {
|
|||
System.out.println("EitherOrElse example");
|
||||
System.out.println();
|
||||
|
||||
ActorSystem application = new ActorSystem("UntypedTransactorExample");
|
||||
ActorSystem application = ActorSystem.create("UntypedTransactorExample");
|
||||
|
||||
final Ref<Integer> left = new Ref<Integer>(100);
|
||||
final Ref<Integer> right = new Ref<Integer>(100);
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ public class RetryExample {
|
|||
System.out.println("Retry example");
|
||||
System.out.println();
|
||||
|
||||
ActorSystem application = new ActorSystem("RetryExample");
|
||||
ActorSystem application = ActorSystem.create("RetryExample");
|
||||
|
||||
final Ref<Double> account1 = new Ref<Double>(100.0);
|
||||
final Ref<Double> account2 = new Ref<Double>(100.0);
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ public class UntypedCoordinatedExample {
|
|||
System.out.println("Untyped transactor example");
|
||||
System.out.println();
|
||||
|
||||
ActorSystem application = new ActorSystem("UntypedCoordinatedExample");
|
||||
ActorSystem application = ActorSystem.create("UntypedCoordinatedExample");
|
||||
|
||||
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));
|
||||
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ public class UntypedTransactorExample {
|
|||
System.out.println("Untyped transactor example");
|
||||
System.out.println();
|
||||
|
||||
ActorSystem application = new ActorSystem("UntypedTransactorExample");
|
||||
ActorSystem application = ActorSystem.create("UntypedTransactorExample");
|
||||
|
||||
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class));
|
||||
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class));
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ import scala.collection.JavaConverters;
|
|||
import scala.collection.Seq;
|
||||
|
||||
public class UntypedCoordinatedIncrementTest {
|
||||
ActorSystem application = new ActorSystem("UntypedCoordinatedIncrementTest");
|
||||
ActorSystem application = ActorSystem.create("UntypedCoordinatedIncrementTest");
|
||||
|
||||
List<ActorRef> counters;
|
||||
ActorRef failer;
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ import scala.collection.JavaConverters;
|
|||
import scala.collection.Seq;
|
||||
|
||||
public class UntypedTransactorTest {
|
||||
ActorSystem application = new ActorSystem("UntypedTransactorTest");
|
||||
ActorSystem application = ActorSystem.create("UntypedTransactorTest");
|
||||
|
||||
List<ActorRef> counters;
|
||||
ActorRef failer;
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ class ConfigSpec extends WordSpec with MustMatchers {
|
|||
|
||||
"The default configuration file (i.e. akka-reference.conf)" should {
|
||||
"contain all configuration properties for akka-stm that are used in code with their correct defaults" in {
|
||||
val config = ActorSystem("ConfigSpec").config
|
||||
val config = ActorSystem("ConfigSpec").AkkaConfig.config
|
||||
|
||||
import config._
|
||||
|
||||
|
|
|
|||
|
|
@ -61,9 +61,9 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll {
|
|||
val numCounters = 4
|
||||
|
||||
def actorOfs = {
|
||||
def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i)))
|
||||
def createCounter(i: Int) = system.actorOf(Props(new Counter("counter" + i)))
|
||||
val counters = (1 to numCounters) map createCounter
|
||||
val failer = app.actorOf(Props(new Failer))
|
||||
val failer = system.actorOf(Props(new Failer))
|
||||
(counters, failer)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -104,9 +104,9 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
|
|||
val numCounters = 2
|
||||
|
||||
def actorOfs = {
|
||||
def createCounter(i: Int) = app.actorOf(Props(new FickleCounter("counter" + i)))
|
||||
def createCounter(i: Int) = system.actorOf(Props(new FickleCounter("counter" + i)))
|
||||
val counters = (1 to numCounters) map createCounter
|
||||
val coordinator = app.actorOf(Props(new Coordinator("coordinator")))
|
||||
val coordinator = system.actorOf(Props(new Coordinator("coordinator")))
|
||||
(counters, coordinator)
|
||||
}
|
||||
|
||||
|
|
@ -116,7 +116,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
|
|||
EventFilter[ExpectedFailureException](),
|
||||
EventFilter[CoordinatedTransactionException](),
|
||||
EventFilter[ActorTimeoutException]())
|
||||
app.eventStream.publish(Mute(ignoreExceptions))
|
||||
system.eventStream.publish(Mute(ignoreExceptions))
|
||||
val (counters, coordinator) = actorOfs
|
||||
val latch = new CountDownLatch(1)
|
||||
coordinator ! FriendlyIncrement(counters, latch)
|
||||
|
|
|
|||
|
|
@ -84,9 +84,9 @@ class TransactorSpec extends AkkaSpec {
|
|||
val numCounters = 3
|
||||
|
||||
def createTransactors = {
|
||||
def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i)))
|
||||
def createCounter(i: Int) = system.actorOf(Props(new Counter("counter" + i)))
|
||||
val counters = (1 to numCounters) map createCounter
|
||||
val failer = app.actorOf(Props(new Failer))
|
||||
val failer = system.actorOf(Props(new Failer))
|
||||
(counters, failer)
|
||||
}
|
||||
|
||||
|
|
@ -124,7 +124,7 @@ class TransactorSpec extends AkkaSpec {
|
|||
|
||||
"Transactor" should {
|
||||
"be usable without overriding normally" in {
|
||||
val transactor = app.actorOf(Props(new Setter))
|
||||
val transactor = system.actorOf(Props(new Setter))
|
||||
val ref = Ref(0)
|
||||
val latch = TestLatch(1)
|
||||
transactor ! Set(ref, 5, latch)
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import akka.event.EventStream
|
|||
* @since 1.1
|
||||
*/
|
||||
class TestActorRef[T <: Actor](
|
||||
_app: ActorSystem,
|
||||
_app: ActorSystemImpl,
|
||||
_deadLetterMailbox: Mailbox,
|
||||
_eventStream: EventStream,
|
||||
_scheduler: Scheduler,
|
||||
|
|
@ -63,10 +63,12 @@ object TestActorRef {
|
|||
|
||||
def apply[T <: Actor](props: Props)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, randomName)
|
||||
|
||||
def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, app.guardian, name)
|
||||
def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] =
|
||||
apply[T](props, app.asInstanceOf[ActorSystemImpl].guardian, name)
|
||||
|
||||
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = {
|
||||
new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name)
|
||||
val impl = app.asInstanceOf[ActorSystemImpl]
|
||||
new TestActorRef(impl, impl.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name)
|
||||
}
|
||||
|
||||
def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName)
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ import akka.event.EventStream
|
|||
* @since 1.2
|
||||
*/
|
||||
class TestFSMRef[S, D, T <: Actor](
|
||||
app: ActorSystem,
|
||||
app: ActorSystemImpl,
|
||||
_deadLetterMailbox: Mailbox,
|
||||
_eventStream: EventStream,
|
||||
_scheduler: Scheduler,
|
||||
|
|
@ -89,9 +89,13 @@ class TestFSMRef[S, D, T <: Actor](
|
|||
|
||||
object TestFSMRef {
|
||||
|
||||
def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
|
||||
new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () ⇒ factory), app.guardian, TestActorRef.randomName)
|
||||
def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = {
|
||||
val impl = app.asInstanceOf[ActorSystemImpl]
|
||||
new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () ⇒ factory), impl.guardian, TestActorRef.randomName)
|
||||
}
|
||||
|
||||
def apply[S, D, T <: Actor](factory: ⇒ T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
|
||||
new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () ⇒ factory), app.guardian, name)
|
||||
def apply[S, D, T <: Actor](factory: ⇒ T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = {
|
||||
val impl = app.asInstanceOf[ActorSystemImpl]
|
||||
new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () ⇒ factory), impl.guardian, name)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -80,7 +80,7 @@ class TestKit(_app: ActorSystem) {
|
|||
|
||||
import TestActor.{ Message, RealMessage, NullMessage }
|
||||
|
||||
implicit val app = _app
|
||||
implicit val system = _app
|
||||
|
||||
private val queue = new LinkedBlockingDeque[Message]()
|
||||
private[akka] var lastMessage: Message = NullMessage
|
||||
|
|
@ -91,9 +91,12 @@ class TestKit(_app: ActorSystem) {
|
|||
* ActorRef of the test actor. Access is provided to enable e.g.
|
||||
* registration as message target.
|
||||
*/
|
||||
val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue))
|
||||
.copy(dispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler)),
|
||||
val testActor: ActorRef = {
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
impl.systemActorOf(Props(new TestActor(queue))
|
||||
.copy(dispatcher = new CallingThreadDispatcher(impl.deadLetterMailbox, impl.eventStream, impl.scheduler)),
|
||||
"testActor" + TestKit.testActorId.incrementAndGet)
|
||||
}
|
||||
|
||||
private var end: Duration = Duration.Undefined
|
||||
|
||||
|
|
@ -124,7 +127,7 @@ class TestKit(_app: ActorSystem) {
|
|||
* block or missing that it returns the properly dilated default for this
|
||||
* case from AkkaConfig (key "akka.test.single-expect-default").
|
||||
*/
|
||||
def remaining: Duration = if (end == Duration.Undefined) app.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now
|
||||
def remaining: Duration = if (end == Duration.Undefined) system.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now
|
||||
|
||||
/**
|
||||
* Query queue status.
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.testkit
|
|||
import akka.config.Configuration
|
||||
import org.scalatest.{ WordSpec, BeforeAndAfterAll }
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
|
|
@ -16,16 +16,16 @@ import akka.dispatch.FutureTimeoutException
|
|||
abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
|
||||
extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll {
|
||||
|
||||
val log: LoggingAdapter = Logging(app.eventStream, this)
|
||||
val log: LoggingAdapter = Logging(system, this)
|
||||
|
||||
final override def beforeAll {
|
||||
atStartup()
|
||||
}
|
||||
|
||||
final override def afterAll {
|
||||
app.stop()
|
||||
try app.terminationFuture.await(5 seconds) catch {
|
||||
case _: FutureTimeoutException ⇒ app.log.warning("failed to stop within 5 seconds")
|
||||
system.stop()
|
||||
try system.asInstanceOf[ActorSystemImpl].terminationFuture.await(5 seconds) catch {
|
||||
case _: FutureTimeoutException ⇒ system.log.warning("failed to stop within 5 seconds")
|
||||
}
|
||||
atTermination()
|
||||
}
|
||||
|
|
@ -34,9 +34,9 @@ abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
|
|||
|
||||
protected def atTermination() {}
|
||||
|
||||
def this(config: Configuration) = this(new ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config))
|
||||
def this(config: Configuration) = this(ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config))
|
||||
|
||||
def actorOf(props: Props): ActorRef = app.actorOf(props)
|
||||
def actorOf(props: Props): ActorRef = system.actorOf(props)
|
||||
|
||||
def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz))
|
||||
|
||||
|
|
|
|||
|
|
@ -170,8 +170,8 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
|
|||
counter = 2
|
||||
|
||||
val boss = TestActorRef(Props(new TActor {
|
||||
|
||||
val ref = new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(new TActor {
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
val ref = new TestActorRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(new TActor {
|
||||
def receiveT = { case _ ⇒ }
|
||||
override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 }
|
||||
override def postRestart(reason: Throwable) { counter -= 1 }
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0)
|
|||
val now = System.nanoTime
|
||||
intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) }
|
||||
val diff = System.nanoTime - now
|
||||
val target = (1000000000l * app.AkkaConfig.TestTimeFactor).toLong
|
||||
val target = (1000000000l * system.AkkaConfig.TestTimeFactor).toLong
|
||||
diff must be > (target - 300000000l)
|
||||
diff must be < (target + 1000000000l)
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue