Removed all 'actorOf' methods that does not take a 'Props', and changed all callers to use 'actorOf(Props(..))'
Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
parent
86a5114d79
commit
c9b787f029
85 changed files with 464 additions and 518 deletions
|
|
@ -26,7 +26,7 @@ public class JavaAPI {
|
|||
|
||||
@Test
|
||||
public void mustBeAbleToCreateActorRefFromClass() {
|
||||
ActorRef ref = system.actorOf(JavaAPITestActor.class);
|
||||
ActorRef ref = system.actorOf(new Props(JavaAPITestActor.class));
|
||||
assertNotNull(ref);
|
||||
}
|
||||
|
||||
|
|
@ -42,7 +42,7 @@ public class JavaAPI {
|
|||
|
||||
@Test
|
||||
public void mustAcceptSingleArgTell() {
|
||||
ActorRef ref = system.actorOf(JavaAPITestActor.class);
|
||||
ActorRef ref = system.actorOf(new Props(JavaAPITestActor.class));
|
||||
ref.tell("hallo");
|
||||
ref.tell("hallo", ref);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -63,16 +63,16 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach w
|
|||
"An Actor" must {
|
||||
|
||||
"reply to bang message using reply" in {
|
||||
val replyActor = system.actorOf[ReplyActor]
|
||||
val senderActor = system.actorOf(new SenderActor(replyActor))
|
||||
val replyActor = system.actorOf(Props[ReplyActor])
|
||||
val senderActor = system.actorOf(Props(new SenderActor(replyActor)))
|
||||
senderActor ! "Init"
|
||||
state.finished.await
|
||||
state.s must be("Reply")
|
||||
}
|
||||
|
||||
"reply to bang message using implicit sender" in {
|
||||
val replyActor = system.actorOf[ReplyActor]
|
||||
val senderActor = system.actorOf(new SenderActor(replyActor))
|
||||
val replyActor = system.actorOf(Props[ReplyActor])
|
||||
val senderActor = system.actorOf(Props(new SenderActor(replyActor)))
|
||||
senderActor ! "InitImplicit"
|
||||
state.finished.await
|
||||
state.s must be("ReplyImplicit")
|
||||
|
|
|
|||
|
|
@ -145,82 +145,82 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
filterException[akka.actor.ActorInitializationException] {
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new Actor {
|
||||
actorOf(Props(new Actor {
|
||||
val nested = promiseIntercept(new Actor { def receive = { case _ ⇒ } })(result)
|
||||
def receive = { case _ ⇒ }
|
||||
}))
|
||||
})))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(promiseIntercept(new FailingOuterActor(actorOf(new InnerActor)))(result)))
|
||||
actorOf(Props(promiseIntercept(new FailingOuterActor(actorOf(Props(new InnerActor))))(result))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new OuterActor(actorOf(promiseIntercept(new FailingInnerActor)(result)))))
|
||||
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept(new FailingInnerActor)(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(promiseIntercept(new FailingInheritingOuterActor(actorOf(new InnerActor)))(result)))
|
||||
actorOf(Props(promiseIntercept(new FailingInheritingOuterActor(actorOf(Props(new InnerActor))))(result))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new FailingOuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result)))))
|
||||
actorOf(Props(new FailingOuterActor(actorOf(Props(promiseIntercept(new FailingInheritingInnerActor)(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new FailingInheritingOuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result)))))
|
||||
actorOf(Props(new FailingInheritingOuterActor(actorOf(Props(promiseIntercept(new FailingInheritingInnerActor)(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new FailingInheritingOuterActor(actorOf(promiseIntercept(new FailingInnerActor)(result)))))
|
||||
actorOf(Props(new FailingInheritingOuterActor(actorOf(Props(promiseIntercept(new FailingInnerActor)(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new OuterActor(actorOf(new InnerActor {
|
||||
actorOf(Props(new OuterActor(actorOf(Props(new InnerActor {
|
||||
val a = promiseIntercept(new InnerActor)(result)
|
||||
}))))
|
||||
}))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new FailingOuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result)))))
|
||||
actorOf(Props(new FailingOuterActor(actorOf(Props(promiseIntercept(new FailingInheritingInnerActor)(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new OuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result)))))
|
||||
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept(new FailingInheritingInnerActor)(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
||||
intercept[akka.actor.ActorInitializationException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new OuterActor(actorOf(promiseIntercept({ new InnerActor; new InnerActor })(result)))))
|
||||
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept({ new InnerActor; new InnerActor })(result)))))))
|
||||
}
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
|
@ -229,7 +229,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
filterException[java.lang.IllegalStateException] {
|
||||
(intercept[java.lang.IllegalStateException] {
|
||||
wrap(result ⇒
|
||||
actorOf(new OuterActor(actorOf(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result)))))
|
||||
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result)))))))
|
||||
}).getMessage must be === "Ur state be b0rked"
|
||||
|
||||
contextStackMustBeEmpty
|
||||
|
|
@ -237,7 +237,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"be serializable using Java Serialization on local node" in {
|
||||
val a = system.actorOf[InnerActor]
|
||||
val a = system.actorOf(Props[InnerActor])
|
||||
|
||||
import java.io._
|
||||
|
||||
|
|
@ -260,7 +260,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"throw an exception on deserialize if no system in scope" in {
|
||||
val a = system.actorOf[InnerActor]
|
||||
val a = system.actorOf(Props[InnerActor])
|
||||
|
||||
import java.io._
|
||||
|
||||
|
|
@ -301,10 +301,10 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"support nested actorOfs" in {
|
||||
val a = system.actorOf(new Actor {
|
||||
val nested = system.actorOf(new Actor { def receive = { case _ ⇒ } })
|
||||
val a = system.actorOf(Props(new Actor {
|
||||
val nested = system.actorOf(Props(new Actor { def receive = { case _ ⇒ } }))
|
||||
def receive = { case _ ⇒ sender ! nested }
|
||||
})
|
||||
}))
|
||||
|
||||
val nested = (a ? "any").as[ActorRef].get
|
||||
a must not be null
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ object Chameneos {
|
|||
var numFaded = 0
|
||||
|
||||
override def preStart() = {
|
||||
for (i ← 0 until numChameneos) context.actorOf(new Chameneo(self, colours(i % 3), i))
|
||||
for (i ← 0 until numChameneos) context.actorOf(Props(new Chameneo(self, colours(i % 3), i)))
|
||||
}
|
||||
|
||||
def receive = {
|
||||
|
|
@ -107,7 +107,7 @@ object Chameneos {
|
|||
def run {
|
||||
// System.setProperty("akka.config", "akka.conf")
|
||||
Chameneos.start = System.currentTimeMillis
|
||||
val system = ActorSystem().actorOf(new Mall(1000000, 4))
|
||||
val system = ActorSystem().actorOf(Props(new Mall(1000000, 4)))
|
||||
Thread.sleep(10000)
|
||||
println("Elapsed: " + (end - start))
|
||||
system.stop()
|
||||
|
|
|
|||
|
|
@ -112,14 +112,14 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
|
|||
import latches._
|
||||
|
||||
// lock that locked after being open for 1 sec
|
||||
val lock = system.actorOf(new Lock("33221", 1 second, latches))
|
||||
val lock = system.actorOf(Props(new Lock("33221", 1 second, latches)))
|
||||
|
||||
val transitionTester = system.actorOf(new Actor {
|
||||
val transitionTester = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Transition(_, _, _) ⇒ transitionCallBackLatch.open
|
||||
case CurrentState(_, Locked) ⇒ initialStateLatch.open
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
lock ! SubscribeTransitionCallBack(transitionTester)
|
||||
initialStateLatch.await
|
||||
|
|
@ -143,13 +143,13 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
|
|||
val answerLatch = TestLatch()
|
||||
object Hello
|
||||
object Bye
|
||||
val tester = system.actorOf(new Actor {
|
||||
val tester = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
case Hello ⇒ lock ! "hello"
|
||||
case "world" ⇒ answerLatch.open
|
||||
case Bye ⇒ lock ! "bye"
|
||||
}
|
||||
})
|
||||
}))
|
||||
tester ! Hello
|
||||
answerLatch.await
|
||||
|
||||
|
|
@ -185,7 +185,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
|
|||
case x ⇒ testActor ! x
|
||||
}
|
||||
}
|
||||
val ref = system.actorOf(fsm)
|
||||
val ref = system.actorOf(Props(fsm))
|
||||
started.await
|
||||
ref.stop()
|
||||
expectMsg(1 second, fsm.StopEvent(Shutdown, 1, null))
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
|||
import FSMTimingSpec._
|
||||
import FSM._
|
||||
|
||||
val fsm = system.actorOf(new StateMachine(testActor))
|
||||
val fsm = system.actorOf(Props(new StateMachine(testActor)))
|
||||
fsm ! SubscribeTransitionCallBack(testActor)
|
||||
expectMsg(1 second, CurrentState(fsm, Initial))
|
||||
|
||||
|
|
|
|||
|
|
@ -56,7 +56,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
|
|||
"A FSM transition notifier" must {
|
||||
|
||||
"notify listeners" in {
|
||||
val fsm = system.actorOf(new MyFSM(testActor))
|
||||
val fsm = system.actorOf(Props(new MyFSM(testActor)))
|
||||
within(1 second) {
|
||||
fsm ! SubscribeTransitionCallBack(testActor)
|
||||
expectMsg(CurrentState(fsm, 0))
|
||||
|
|
@ -68,8 +68,8 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
|
||||
"not fail when listener goes away" in {
|
||||
val forward = system.actorOf(new Forwarder(testActor))
|
||||
val fsm = system.actorOf(new MyFSM(testActor))
|
||||
val forward = system.actorOf(Props(new Forwarder(testActor)))
|
||||
val fsm = system.actorOf(Props(new MyFSM(testActor)))
|
||||
val sup = system.actorOf(Props(new Actor {
|
||||
context.watch(fsm)
|
||||
def receive = { case _ ⇒ }
|
||||
|
|
@ -88,7 +88,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
|
|||
"A FSM" must {
|
||||
|
||||
"make previous and next state data available in onTransition" in {
|
||||
val fsm = system.actorOf(new OtherFSM(testActor))
|
||||
val fsm = system.actorOf(Props(new OtherFSM(testActor)))
|
||||
within(300 millis) {
|
||||
fsm ! "tick"
|
||||
expectMsg((0, 1))
|
||||
|
|
|
|||
|
|
@ -13,14 +13,14 @@ object ForwardActorSpec {
|
|||
val ExpectedMessage = "FOO"
|
||||
|
||||
def createForwardingChain(system: ActorSystem): ActorRef = {
|
||||
val replier = system.actorOf(new Actor {
|
||||
val replier = system.actorOf(Props(new Actor {
|
||||
def receive = { case x ⇒ sender ! x }
|
||||
})
|
||||
}))
|
||||
|
||||
def mkforwarder(forwardTo: ActorRef) = system.actorOf(
|
||||
def mkforwarder(forwardTo: ActorRef) = system.actorOf(Props(
|
||||
new Actor {
|
||||
def receive = { case x ⇒ forwardTo forward x }
|
||||
})
|
||||
}))
|
||||
|
||||
mkforwarder(mkforwarder(mkforwarder(replier)))
|
||||
}
|
||||
|
|
@ -35,7 +35,7 @@ class ForwardActorSpec extends AkkaSpec {
|
|||
"forward actor reference when invoking forward on bang" in {
|
||||
val latch = new TestLatch(1)
|
||||
|
||||
val replyTo = system.actorOf(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } })
|
||||
val replyTo = system.actorOf(Props(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } }))
|
||||
|
||||
val chain = createForwardingChain(system)
|
||||
|
||||
|
|
|
|||
|
|
@ -12,12 +12,12 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
|||
"An Actor" must {
|
||||
|
||||
"be able to hotswap its behavior with become(..)" in {
|
||||
val a = system.actorOf(new Actor {
|
||||
val a = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "init" ⇒ sender ! "init"
|
||||
case "swap" ⇒ context.become({ case x: String ⇒ context.sender ! x })
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
a ! "init"
|
||||
expectMsg("init")
|
||||
|
|
@ -27,7 +27,7 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
|||
}
|
||||
|
||||
"be able to revert hotswap its behavior with unbecome" in {
|
||||
val a = system.actorOf(new Actor {
|
||||
val a = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "init" ⇒ sender ! "init"
|
||||
case "swap" ⇒
|
||||
|
|
@ -38,7 +38,7 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
|||
context.unbecome()
|
||||
})
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
a ! "init"
|
||||
expectMsg("init")
|
||||
|
|
@ -54,7 +54,7 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
|||
|
||||
"revert to initial state on restart" in {
|
||||
|
||||
val a = system.actorOf(new Actor {
|
||||
val a = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "state" ⇒ sender ! "0"
|
||||
case "swap" ⇒
|
||||
|
|
@ -65,7 +65,7 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
|||
})
|
||||
sender ! "swapped"
|
||||
}
|
||||
})
|
||||
}))
|
||||
a ! "state"
|
||||
expectMsg("0")
|
||||
a ! "swap"
|
||||
|
|
|
|||
|
|
@ -46,7 +46,7 @@ object IOActorSpec {
|
|||
class SimpleEchoClient(host: String, port: Int, ioManager: ActorRef) extends Actor with IO {
|
||||
|
||||
lazy val socket: SocketHandle = connect(ioManager, host, port)(reader)
|
||||
lazy val reader: ActorRef = context.actorOf {
|
||||
lazy val reader: ActorRef = context.actorOf(Props({
|
||||
new Actor with IO {
|
||||
def receiveIO = {
|
||||
case length: Int ⇒
|
||||
|
|
@ -54,7 +54,7 @@ object IOActorSpec {
|
|||
sender ! bytes
|
||||
}
|
||||
}
|
||||
}
|
||||
}))
|
||||
|
||||
def receiveIO = {
|
||||
case bytes: ByteString ⇒
|
||||
|
|
@ -186,10 +186,10 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {
|
|||
"an IO Actor" must {
|
||||
"run echo server" in {
|
||||
val started = TestLatch(1)
|
||||
val ioManager = system.actorOf(new IOManager(2)) // teeny tiny buffer
|
||||
val server = system.actorOf(new SimpleEchoServer("localhost", 8064, ioManager, started))
|
||||
val ioManager = system.actorOf(Props(new IOManager(2))) // teeny tiny buffer
|
||||
val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064, ioManager, started)))
|
||||
started.await
|
||||
val client = system.actorOf(new SimpleEchoClient("localhost", 8064, ioManager))
|
||||
val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064, ioManager)))
|
||||
val f1 = client ? ByteString("Hello World!1")
|
||||
val f2 = client ? ByteString("Hello World!2")
|
||||
val f3 = client ? ByteString("Hello World!3")
|
||||
|
|
@ -203,10 +203,10 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {
|
|||
|
||||
"run echo server under high load" in {
|
||||
val started = TestLatch(1)
|
||||
val ioManager = system.actorOf(new IOManager())
|
||||
val server = system.actorOf(new SimpleEchoServer("localhost", 8065, ioManager, started))
|
||||
val ioManager = system.actorOf(Props(new IOManager()))
|
||||
val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065, ioManager, started)))
|
||||
started.await
|
||||
val client = system.actorOf(new SimpleEchoClient("localhost", 8065, ioManager))
|
||||
val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065, ioManager)))
|
||||
val list = List.range(0, 1000)
|
||||
val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString))
|
||||
assert(f.get.size === 1000)
|
||||
|
|
@ -217,10 +217,10 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {
|
|||
|
||||
"run echo server under high load with small buffer" in {
|
||||
val started = TestLatch(1)
|
||||
val ioManager = system.actorOf(new IOManager(2))
|
||||
val server = system.actorOf(new SimpleEchoServer("localhost", 8066, ioManager, started))
|
||||
val ioManager = system.actorOf(Props(new IOManager(2)))
|
||||
val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8066, ioManager, started)))
|
||||
started.await
|
||||
val client = system.actorOf(new SimpleEchoClient("localhost", 8066, ioManager))
|
||||
val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8066, ioManager)))
|
||||
val list = List.range(0, 1000)
|
||||
val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString))
|
||||
assert(f.get.size === 1000)
|
||||
|
|
@ -231,11 +231,11 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {
|
|||
|
||||
"run key-value store" in {
|
||||
val started = TestLatch(1)
|
||||
val ioManager = system.actorOf(new IOManager(2)) // teeny tiny buffer
|
||||
val server = system.actorOf(new KVStore("localhost", 8067, ioManager, started))
|
||||
val ioManager = system.actorOf(Props(new IOManager(2))) // teeny tiny buffer
|
||||
val server = system.actorOf(Props(new KVStore("localhost", 8067, ioManager, started)))
|
||||
started.await
|
||||
val client1 = system.actorOf(new KVClient("localhost", 8067, ioManager))
|
||||
val client2 = system.actorOf(new KVClient("localhost", 8067, ioManager))
|
||||
val client1 = system.actorOf(Props(new KVClient("localhost", 8067, ioManager)))
|
||||
val client2 = system.actorOf(Props(new KVClient("localhost", 8067, ioManager)))
|
||||
val f1 = client1 ? (('set, "hello", ByteString("World")))
|
||||
val f2 = client1 ? (('set, "test", ByteString("No one will read me")))
|
||||
val f3 = client1 ? (('get, "hello"))
|
||||
|
|
|
|||
|
|
@ -42,12 +42,12 @@ class LocalActorRefProviderSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"only create one instance of an actor from within the same message invocation" in {
|
||||
val supervisor = system.actorOf(new Actor {
|
||||
val supervisor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "" ⇒
|
||||
val a, b = context.actorOf(Props.empty, "duplicate")
|
||||
}
|
||||
})
|
||||
}))
|
||||
EventFilter[InvalidActorNameException](occurrences = 1) intercept {
|
||||
supervisor ! ""
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,13 +17,13 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
"get timeout" in {
|
||||
val timeoutLatch = TestLatch()
|
||||
|
||||
val timeoutActor = system.actorOf(new Actor {
|
||||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
timeoutLatch.await
|
||||
timeoutActor.stop()
|
||||
|
|
@ -33,14 +33,14 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutLatch = TestLatch()
|
||||
case object Tick
|
||||
|
||||
val timeoutActor = system.actorOf(new Actor {
|
||||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
case Tick ⇒ ()
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
timeoutActor ! Tick
|
||||
|
||||
|
|
@ -53,7 +53,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutLatch = TestLatch()
|
||||
case object Tick
|
||||
|
||||
val timeoutActor = system.actorOf(new Actor {
|
||||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
|
|
@ -63,7 +63,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
timeoutLatch.open
|
||||
context.resetReceiveTimeout()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
timeoutActor ! Tick
|
||||
|
||||
|
|
@ -75,11 +75,11 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
"not receive timeout message when not specified" in {
|
||||
val timeoutLatch = TestLatch()
|
||||
|
||||
val timeoutActor = system.actorOf(new Actor {
|
||||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
timeoutLatch.awaitTimeout(1 second) // timeout expected
|
||||
timeoutActor.stop()
|
||||
|
|
|
|||
|
|
@ -26,9 +26,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
"schedule more than once" in {
|
||||
case object Tick
|
||||
val countDownLatch = new CountDownLatch(3)
|
||||
val tickActor = system.actorOf(new Actor {
|
||||
val tickActor = system.actorOf(Props(new Actor {
|
||||
def receive = { case Tick ⇒ countDownLatch.countDown() }
|
||||
})
|
||||
}))
|
||||
// run every 50 milliseconds
|
||||
collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds, tickActor, Tick))
|
||||
|
||||
|
|
@ -56,9 +56,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
"schedule once" in {
|
||||
case object Tick
|
||||
val countDownLatch = new CountDownLatch(3)
|
||||
val tickActor = system.actorOf(new Actor {
|
||||
val tickActor = system.actorOf(Props(new Actor {
|
||||
def receive = { case Tick ⇒ countDownLatch.countDown() }
|
||||
})
|
||||
}))
|
||||
|
||||
// run after 300 millisec
|
||||
collectCancellable(system.scheduler.scheduleOnce(300 milliseconds, tickActor, Tick))
|
||||
|
|
@ -81,9 +81,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
object Ping
|
||||
val ticks = new CountDownLatch(1)
|
||||
|
||||
val actor = system.actorOf(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = { case Ping ⇒ ticks.countDown() }
|
||||
})
|
||||
}))
|
||||
|
||||
(1 to 10).foreach { i ⇒
|
||||
val timeout = collectCancellable(system.scheduler.scheduleOnce(1 second, actor, Ping))
|
||||
|
|
@ -131,7 +131,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
|
||||
case class Msg(ts: Long)
|
||||
|
||||
val actor = system.actorOf(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Msg(ts) ⇒
|
||||
val now = System.nanoTime
|
||||
|
|
@ -139,7 +139,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
if (now - ts < 10000000) throw new RuntimeException("Interval is too small: " + (now - ts))
|
||||
ticks.countDown()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
(1 to 300).foreach { i ⇒
|
||||
collectCancellable(system.scheduler.scheduleOnce(10 milliseconds, actor, Msg(System.nanoTime)))
|
||||
|
|
@ -154,11 +154,11 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
|
||||
case object Msg
|
||||
|
||||
val actor = system.actorOf(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Msg ⇒ ticks.countDown()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val startTime = System.nanoTime()
|
||||
val cancellable = system.scheduler.schedule(1 second, 100 milliseconds, actor, Msg)
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
package akka.actor.dispatch
|
||||
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import akka.actor.Actor
|
||||
import akka.actor._
|
||||
import akka.testkit.AkkaSpec
|
||||
|
||||
/**
|
||||
|
|
@ -31,8 +31,8 @@ class DispatcherActorsSpec extends AkkaSpec {
|
|||
"not block fast actors by slow actors" in {
|
||||
val sFinished = new CountDownLatch(50)
|
||||
val fFinished = new CountDownLatch(10)
|
||||
val s = system.actorOf(new SlowActor(sFinished))
|
||||
val f = system.actorOf(new FastActor(fFinished))
|
||||
val s = system.actorOf(Props(new SlowActor(sFinished)))
|
||||
val f = system.actorOf(Props(new FastActor(fFinished)))
|
||||
|
||||
// send a lot of stuff to s
|
||||
for (i ← 1 to 50) {
|
||||
|
|
|
|||
|
|
@ -16,13 +16,13 @@ class ListenerSpec extends AkkaSpec {
|
|||
val barLatch = TestLatch(2)
|
||||
val barCount = new AtomicInteger(0)
|
||||
|
||||
val broadcast = system.actorOf(new Actor with Listeners {
|
||||
val broadcast = system.actorOf(Props(new Actor with Listeners {
|
||||
def receive = listenerManagement orElse {
|
||||
case "foo" ⇒ gossip("bar")
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
def newListener = system.actorOf(new Actor {
|
||||
def newListener = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "bar" ⇒
|
||||
barCount.incrementAndGet
|
||||
|
|
@ -30,7 +30,7 @@ class ListenerSpec extends AkkaSpec {
|
|||
case "foo" ⇒
|
||||
fooLatch.countDown()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val a1 = newListener
|
||||
val a2 = newListener
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ import org.scalacheck._
|
|||
import org.scalacheck.Arbitrary._
|
||||
import org.scalacheck.Prop._
|
||||
import org.scalacheck.Gen._
|
||||
import akka.actor.{ Actor, ActorRef, Status }
|
||||
import akka.actor._
|
||||
import akka.testkit.{ EventFilter, filterEvents, filterException }
|
||||
import akka.util.duration._
|
||||
import org.multiverse.api.latches.StandardLatch
|
||||
|
|
@ -116,7 +116,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"from an Actor" that {
|
||||
"returns a result" must {
|
||||
behave like futureWithResult { test ⇒
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
val future = actor ? "Hello"
|
||||
future.await
|
||||
test(future, "World")
|
||||
|
|
@ -126,7 +126,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"throws an exception" must {
|
||||
behave like futureWithException[RuntimeException] { test ⇒
|
||||
filterException[RuntimeException] {
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
val future = actor ? "Failure"
|
||||
future.await
|
||||
test(future, "Expected exception; to test fault-tolerance")
|
||||
|
|
@ -139,8 +139,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"using flatMap with an Actor" that {
|
||||
"will return a result" must {
|
||||
behave like futureWithResult { test ⇒
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor2 = system.actorOf(new Actor { def receive = { case s: String ⇒ sender ! s.toUpperCase } })
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
val actor2 = system.actorOf(Props(new Actor { def receive = { case s: String ⇒ sender ! s.toUpperCase } }))
|
||||
val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s }
|
||||
future.await
|
||||
test(future, "WORLD")
|
||||
|
|
@ -151,8 +151,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"will throw an exception" must {
|
||||
behave like futureWithException[ArithmeticException] { test ⇒
|
||||
filterException[ArithmeticException] {
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor2 = system.actorOf(new Actor { def receive = { case s: String ⇒ sender ! Status.Failure(new ArithmeticException("/ by zero")) } })
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
val actor2 = system.actorOf(Props(new Actor { def receive = { case s: String ⇒ sender ! Status.Failure(new ArithmeticException("/ by zero")) } }))
|
||||
val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s }
|
||||
future.await
|
||||
test(future, "/ by zero")
|
||||
|
|
@ -164,8 +164,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"will throw a MatchError when matching wrong type" must {
|
||||
behave like futureWithException[MatchError] { test ⇒
|
||||
filterException[MatchError] {
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor2 = system.actorOf(new Actor { def receive = { case s: String ⇒ sender ! s.toUpperCase } })
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
val actor2 = system.actorOf(Props(new Actor { def receive = { case s: String ⇒ sender ! s.toUpperCase } }))
|
||||
val future = actor1 ? "Hello" flatMap { case i: Int ⇒ actor2 ? i }
|
||||
future.await
|
||||
test(future, "World (of class java.lang.String)")
|
||||
|
|
@ -180,12 +180,12 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
"compose with for-comprehensions" in {
|
||||
filterException[ClassCastException] {
|
||||
val actor = system.actorOf(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case s: String ⇒ sender ! s.length
|
||||
case i: Int ⇒ sender ! (i * 2).toString
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val future0 = actor ? "Hello"
|
||||
|
||||
|
|
@ -212,12 +212,12 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
filterException[MatchError] {
|
||||
case class Req[T](req: T)
|
||||
case class Res[T](res: T)
|
||||
val actor = system.actorOf(new Actor {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Req(s: String) ⇒ sender ! Res(s.length)
|
||||
case Req(i: Int) ⇒ sender ! Res((i * 2).toString)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val future1 = for {
|
||||
Res(a: Int) ← actor ? Req("Hello")
|
||||
|
|
@ -257,7 +257,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
val future7 = future3 recover { case e: ArithmeticException ⇒ "You got ERROR" }
|
||||
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
|
||||
val future8 = actor ? "Failure"
|
||||
val future9 = actor ? "Failure" recover {
|
||||
|
|
@ -300,9 +300,9 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
"fold" in {
|
||||
val actors = (1 to 10).toList map { _ ⇒
|
||||
system.actorOf(new Actor {
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); sender.tell(add) }
|
||||
})
|
||||
}))
|
||||
}
|
||||
val timeout = 10000
|
||||
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), timeout).mapTo[Int] }
|
||||
|
|
@ -311,9 +311,9 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
"fold by composing" in {
|
||||
val actors = (1 to 10).toList map { _ ⇒
|
||||
system.actorOf(new Actor {
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); sender.tell(add) }
|
||||
})
|
||||
}))
|
||||
}
|
||||
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), 10000).mapTo[Int] }
|
||||
futures.foldLeft(Future(0))((fr, fa) ⇒ for (r ← fr; a ← fa) yield (r + a)).get must be(45)
|
||||
|
|
@ -322,14 +322,14 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"fold with an exception" in {
|
||||
filterException[IllegalArgumentException] {
|
||||
val actors = (1 to 10).toList map { _ ⇒
|
||||
system.actorOf(new Actor {
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case (add: Int, wait: Int) ⇒
|
||||
Thread.sleep(wait)
|
||||
if (add == 6) sender ! Status.Failure(new IllegalArgumentException("shouldFoldResultsWithException: expected"))
|
||||
else sender.tell(add)
|
||||
}
|
||||
})
|
||||
}))
|
||||
}
|
||||
val timeout = 10000
|
||||
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 100), timeout).mapTo[Int] }
|
||||
|
|
@ -358,9 +358,9 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
"shouldReduceResults" in {
|
||||
val actors = (1 to 10).toList map { _ ⇒
|
||||
system.actorOf(new Actor {
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); sender.tell(add) }
|
||||
})
|
||||
}))
|
||||
}
|
||||
val timeout = 10000
|
||||
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), timeout).mapTo[Int] }
|
||||
|
|
@ -370,14 +370,14 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"shouldReduceResultsWithException" in {
|
||||
filterException[IllegalArgumentException] {
|
||||
val actors = (1 to 10).toList map { _ ⇒
|
||||
system.actorOf(new Actor {
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case (add: Int, wait: Int) ⇒
|
||||
Thread.sleep(wait)
|
||||
if (add == 6) sender ! Status.Failure(new IllegalArgumentException("shouldFoldResultsWithException: expected"))
|
||||
else sender.tell(add)
|
||||
}
|
||||
})
|
||||
}))
|
||||
}
|
||||
val timeout = 10000
|
||||
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 100), timeout).mapTo[Int] }
|
||||
|
|
@ -393,21 +393,21 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
"receiveShouldExecuteOnComplete" in {
|
||||
val latch = new StandardLatch
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
actor ? "Hello" onResult { case "World" ⇒ latch.open }
|
||||
assert(latch.tryAwait(5, TimeUnit.SECONDS))
|
||||
actor.stop()
|
||||
}
|
||||
|
||||
"shouldTraverseFutures" in {
|
||||
val oddActor = system.actorOf(new Actor {
|
||||
val oddActor = system.actorOf(Props(new Actor {
|
||||
var counter = 1
|
||||
def receive = {
|
||||
case 'GetNext ⇒
|
||||
sender ! counter
|
||||
counter += 2
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val oddFutures = List.fill(100)(oddActor ? 'GetNext mapTo manifest[Int])
|
||||
assert(Future.sequence(oddFutures).get.sum === 10000)
|
||||
|
|
@ -461,7 +461,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
"futureComposingWithContinuations" in {
|
||||
import Future.flow
|
||||
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
|
||||
val x = Future("Hello")
|
||||
val y = x flatMap (actor ? _) mapTo manifest[String]
|
||||
|
|
@ -490,7 +490,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
filterException[ClassCastException] {
|
||||
import Future.flow
|
||||
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
|
||||
val x = Future(3)
|
||||
val y = (actor ? "Hello").mapTo[Int]
|
||||
|
|
@ -505,7 +505,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
filterException[ClassCastException] {
|
||||
import Future.flow
|
||||
|
||||
val actor = system.actorOf[TestActor]
|
||||
val actor = system.actorOf(Props[TestActor])
|
||||
|
||||
val x = Future("Hello")
|
||||
val y = actor ? "Hello" mapTo manifest[Nothing]
|
||||
|
|
|
|||
|
|
@ -58,11 +58,11 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
|
|||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val clients = (for (i ← 0 until numberOfClients) yield {
|
||||
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))
|
||||
val destination = system.actorOf(Props[Destination])
|
||||
val w4 = system.actorOf(Props(new Waypoint(destination)))
|
||||
val w3 = system.actorOf(Props(new Waypoint(w4)))
|
||||
val w2 = system.actorOf(Props(new Waypoint(w3)))
|
||||
val w1 = system.actorOf(Props(new Waypoint(w2)))
|
||||
Props(new Client(w1, latch, repeatsPerClient, clientDelay.toMicros.intValue, stat)).withDispatcher(clientDispatcher)
|
||||
}).toList.map(system.actorOf(_))
|
||||
|
||||
|
|
@ -133,4 +133,4 @@ object TellLatencyPerformanceSpec {
|
|||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -82,11 +82,11 @@ class ActorPoolSpec extends AkkaSpec with DefaultTimeout {
|
|||
}).withFaultHandler(faultHandler))
|
||||
|
||||
val successes = TestLatch(2)
|
||||
val successCounter = system.actorOf(new Actor {
|
||||
val successCounter = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "success" ⇒ successes.countDown()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
implicit val replyTo = successCounter
|
||||
pool ! "a"
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
"direct router" must {
|
||||
"be started when constructed" in {
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -39,12 +39,12 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(1)
|
||||
|
||||
val counter = new AtomicInteger(0)
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case _ ⇒ counter.incrementAndGet
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val routedActor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -60,12 +60,12 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(1)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -82,7 +82,7 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
"round robin router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -104,12 +104,12 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
for (i ← 0 until connectionCount) {
|
||||
counters = counters :+ new AtomicInteger()
|
||||
|
||||
val connection = system.actorOf(new Actor {
|
||||
val connection = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counters.get(i).get.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
connections = connections :+ connection
|
||||
}
|
||||
|
||||
|
|
@ -138,20 +138,20 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val connection2 = system.actorOf(new Actor {
|
||||
val connection2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -169,12 +169,12 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(1)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case _ ⇒ counter1.incrementAndGet()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -191,7 +191,7 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
"be started when constructed" in {
|
||||
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -202,20 +202,20 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val connection2 = system.actorOf(new Actor {
|
||||
val connection2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -233,12 +233,12 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(1)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case _ ⇒ counter1.incrementAndGet()
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1)))
|
||||
val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
|
||||
|
|
@ -326,12 +326,12 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
for (i ← 0 until connectionCount) {
|
||||
counters = counters :+ new AtomicInteger()
|
||||
|
||||
val connection = system.actorOf(new Actor {
|
||||
val connection = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counters.get(i).get.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
connections = connections :+ connection
|
||||
}
|
||||
|
||||
|
|
@ -359,20 +359,20 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val connection2 = system.actorOf(new Actor {
|
||||
val connection2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
|
||||
|
|
@ -389,7 +389,7 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
case class Stop(id: Option[Int] = None)
|
||||
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(new Actor {
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Stop(None) ⇒ self.stop()
|
||||
case Stop(Some(_id)) if (_id == id) ⇒ self.stop()
|
||||
|
|
@ -400,13 +400,13 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
override def postStop = {
|
||||
shudownLatch foreach (_.countDown())
|
||||
}
|
||||
})
|
||||
}))
|
||||
}
|
||||
|
||||
"broadcast router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val actor1 = system.actorOf[TestActor]
|
||||
val actor1 = system.actorOf(Props[TestActor])
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new BroadcastRouter, connectionManager = new LocalConnectionManager(List(actor1)))
|
||||
val actor = new RoutedActorRef(system, props, system.asInstanceOf[ActorSystemImpl].guardian, "foo")
|
||||
|
|
@ -417,20 +417,20 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val connection2 = system.actorOf(new Actor {
|
||||
val connection2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new BroadcastRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
val actor = new RoutedActorRef(system, props, system.asInstanceOf[ActorSystemImpl].guardian, "foo")
|
||||
|
|
@ -448,22 +448,22 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
|
|||
val doneLatch = new CountDownLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val connection1 = system.actorOf(new Actor {
|
||||
val connection1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒
|
||||
counter1.addAndGet(msg)
|
||||
sender ! "ack"
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val connection2 = system.actorOf(new Actor {
|
||||
val connection2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
})
|
||||
}))
|
||||
|
||||
val props = RoutedProps(routerFactory = () ⇒ new BroadcastRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
|
||||
val actor = new RoutedActorRef(system, props, system.asInstanceOf[ActorSystemImpl].guardian, "foo")
|
||||
|
|
|
|||
|
|
@ -28,16 +28,16 @@ import java.io.{ NotSerializableException, ObjectOutputStream }
|
|||
* context.actorOf(props)
|
||||
*
|
||||
* // Scala
|
||||
* context.actorOf[MyActor]("name")
|
||||
* context.actorOf[MyActor]
|
||||
* context.actorOf(new MyActor(...))
|
||||
* context.actorOf(Props[MyActor]("name")
|
||||
* context.actorOf(Props[MyActor]
|
||||
* context.actorOf(Props(new MyActor(...))
|
||||
*
|
||||
* // Java
|
||||
* context.actorOf(classOf[MyActor]);
|
||||
* context.actorOf(new Creator<MyActor>() {
|
||||
* context.actorOf(Props(new Creator<MyActor>() {
|
||||
* public MyActor create() { ... }
|
||||
* });
|
||||
* context.actorOf(new Creator<MyActor>() {
|
||||
* context.actorOf(Props(new Creator<MyActor>() {
|
||||
* public MyActor create() { ... }
|
||||
* }, "name");
|
||||
* }}}
|
||||
|
|
|
|||
|
|
@ -25,21 +25,21 @@ import scala.annotation.tailrec
|
|||
* <pre>
|
||||
* import Actor._
|
||||
*
|
||||
* val actor = actorOf[MyActor]
|
||||
* val actor = actorOf(Props[MyActor]
|
||||
* actor ! message
|
||||
* actor.stop()
|
||||
* </pre>
|
||||
*
|
||||
* You can also create and start actors like this:
|
||||
* <pre>
|
||||
* val actor = actorOf[MyActor]
|
||||
* val actor = actorOf(Props[MyActor]
|
||||
* </pre>
|
||||
*
|
||||
* Here is an example on how to create an actor with a non-default constructor.
|
||||
* <pre>
|
||||
* import Actor._
|
||||
*
|
||||
* val actor = actorOf(new MyActor(...))
|
||||
* val actor = actorOf(Props(new MyActor(...))
|
||||
* actor ! message
|
||||
* actor.stop()
|
||||
* </pre>
|
||||
|
|
|
|||
|
|
@ -164,82 +164,6 @@ trait ActorRefFactory {
|
|||
*/
|
||||
def actorOf(props: Props, name: String): ActorRef
|
||||
|
||||
/**
|
||||
* Create new actor of the given type as child of this context and give it an automatically
|
||||
* generated name (currently similar to base64-encoded integer count,
|
||||
* reversed and with “$” prepended, may change in the future). The type must have
|
||||
* a no-arg constructor which will be invoked using reflection.
|
||||
*
|
||||
* When invoked on ActorSystem, this method sends a message to the guardian
|
||||
* actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in
|
||||
* the `reference.conf`.
|
||||
*/
|
||||
def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]))
|
||||
|
||||
/**
|
||||
* Create new actor of the given type as child of this context with the given name, which must
|
||||
* not be null, empty or start with “$”. If the given name is already in use,
|
||||
* and `InvalidActorNameException` is thrown. The type must have
|
||||
* a no-arg constructor which will be invoked using reflection.
|
||||
*
|
||||
* When invoked on ActorSystem, this method sends a message to the guardian
|
||||
* actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in
|
||||
* the `reference.conf`.
|
||||
*/
|
||||
def actorOf[T <: Actor](name: String)(implicit m: Manifest[T]): ActorRef =
|
||||
actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]), name)
|
||||
|
||||
/**
|
||||
* Create new actor of the given class as child of this context and give it an automatically
|
||||
* generated name (currently similar to base64-encoded integer count,
|
||||
* reversed and with “$” prepended, may change in the future). The class must have
|
||||
* a no-arg constructor which will be invoked using reflection.
|
||||
*
|
||||
* When invoked on ActorSystem, this method sends a message to the guardian
|
||||
* actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in
|
||||
* the `reference.conf`.
|
||||
*/
|
||||
def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz))
|
||||
|
||||
/**
|
||||
* Create new actor as child of this context and give it an automatically
|
||||
* generated name (currently similar to base64-encoded integer count,
|
||||
* reversed and with “$” prepended, may change in the future). Use this
|
||||
* method to pass constructor arguments to the [[akka.actor.Actor]] while using
|
||||
* only default [[akka.actor.Props]]; otherwise refer to `actorOf(Props)`.
|
||||
*
|
||||
* When invoked on ActorSystem, this method sends a message to the guardian
|
||||
* actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in
|
||||
* the `reference.conf`.
|
||||
*/
|
||||
def actorOf(factory: ⇒ Actor): ActorRef = actorOf(Props(() ⇒ factory))
|
||||
|
||||
/**
|
||||
* ''Java API'': Create new actor as child of this context and give it an
|
||||
* automatically generated name (currently similar to base64-encoded integer
|
||||
* count, reversed and with “$” prepended, may change in the future).
|
||||
*
|
||||
* Identical to `actorOf(Props(() => creator.create()))`.
|
||||
*
|
||||
* When invoked on ActorSystem, this method sends a message to the guardian
|
||||
* actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in
|
||||
* the `reference.conf`.
|
||||
*/
|
||||
def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() ⇒ creator.create()))
|
||||
|
||||
/**
|
||||
* ''Java API'': Create new actor as child of this context with the given name, which must
|
||||
* not be null, empty or start with “$”. If the given name is already in use,
|
||||
* and `InvalidActorNameException` is thrown.
|
||||
*
|
||||
* Identical to `actorOf(Props(() => creator.create()), name)`.
|
||||
*
|
||||
* When invoked on ActorSystem, this method sends a message to the guardian
|
||||
* actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in
|
||||
* the `reference.conf`.
|
||||
*/
|
||||
def actorOf(creator: UntypedActorFactory, name: String): ActorRef = actorOf(Props(() ⇒ creator.create()), name)
|
||||
|
||||
/**
|
||||
* Look-up an actor by path; if it does not exist, returns a reference to
|
||||
* the dead-letter mailbox of the [[akka.actor.ActorSystem]]. If the path
|
||||
|
|
|
|||
|
|
@ -167,16 +167,16 @@ object ActorSystem {
|
|||
* system.actorOf(props)
|
||||
*
|
||||
* // Scala
|
||||
* system.actorOf[MyActor]("name")
|
||||
* system.actorOf[MyActor]
|
||||
* system.actorOf(new MyActor(...))
|
||||
* system.actorOf(Props[MyActor]("name")
|
||||
* system.actorOf(Props[MyActor]
|
||||
* system.actorOf(Props(new MyActor(...))
|
||||
*
|
||||
* // Java
|
||||
* system.actorOf(classOf[MyActor]);
|
||||
* system.actorOf(new Creator<MyActor>() {
|
||||
* system.actorOf(Props(new Creator<MyActor>() {
|
||||
* public MyActor create() { ... }
|
||||
* });
|
||||
* system.actorOf(new Creator<MyActor>() {
|
||||
* system.actorOf(Props(new Creator<MyActor>() {
|
||||
* public MyActor create() { ... }
|
||||
* }, "name");
|
||||
* }}}
|
||||
|
|
|
|||
|
|
@ -81,6 +81,25 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
|
|||
@transient dispatcher: MessageDispatcher = Props.defaultDispatcher,
|
||||
timeout: Timeout = Props.defaultTimeout,
|
||||
faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler) {
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(factory: UntypedActorFactory) = this(
|
||||
creator = () ⇒ factory.create(),
|
||||
dispatcher = Props.defaultDispatcher,
|
||||
timeout = Props.defaultTimeout,
|
||||
faultHandler = Props.defaultFaultHandler)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(actorClass: Class[_ <: Actor]) = this(
|
||||
creator = () ⇒ actorClass.newInstance,
|
||||
dispatcher = Props.defaultDispatcher,
|
||||
timeout = Props.defaultTimeout,
|
||||
faultHandler = Props.defaultFaultHandler)
|
||||
|
||||
/**
|
||||
* No-args constructor that sets all the default values
|
||||
* Java API
|
||||
|
|
|
|||
|
|
@ -21,10 +21,10 @@ class TypedConsumerPublishRequestorTest extends JUnitSuite {
|
|||
|
||||
@Before
|
||||
def setUp{
|
||||
publisher = actorOf(new TypedConsumerPublisherMock)
|
||||
requestor = actorOf(new TypedConsumerPublishRequestor)
|
||||
publisher = actorOf(Props(new TypedConsumerPublisherMock)
|
||||
requestor = actorOf(Props(new TypedConsumerPublishRequestor)
|
||||
requestor ! InitPublishRequestor(publisher)
|
||||
consumer = actorOf(new Actor with Consumer {
|
||||
consumer = actorOf(Props(new Actor with Consumer {
|
||||
def endpointUri = "mock:test"
|
||||
protected def receive = null
|
||||
})
|
||||
|
|
|
|||
|
|
@ -18,10 +18,10 @@ class ConsumerPublishRequestorTest extends JUnitSuite {
|
|||
|
||||
@Before
|
||||
def setUp{
|
||||
publisher = actorOf(new ConsumerPublisherMock)
|
||||
requestor = actorOf(new ConsumerPublishRequestor)
|
||||
publisher = actorOf(Props(new ConsumerPublisherMock)
|
||||
requestor = actorOf(Props(new ConsumerPublishRequestor)
|
||||
requestor ! InitPublishRequestor(publisher)
|
||||
consumer = actorOf(new Actor with Consumer {
|
||||
consumer = actorOf(Props(new Actor with Consumer {
|
||||
def endpointUri = "mock:test"
|
||||
protected def receive = null
|
||||
}).asInstanceOf[LocalActorRef]
|
||||
|
|
|
|||
|
|
@ -9,21 +9,21 @@ class ConsumerRegisteredTest extends JUnitSuite {
|
|||
|
||||
@Test
|
||||
def shouldCreateSomeNonBlockingPublishRequestFromConsumer = {
|
||||
val c = Actor.actorOf[ConsumerActor1]
|
||||
val c = Actor.actorOf(Props[ConsumerActor1]
|
||||
val event = ConsumerActorRegistered.eventFor(c)
|
||||
assert(event === Some(ConsumerActorRegistered(c, consumerOf(c))))
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldCreateSomeBlockingPublishRequestFromConsumer = {
|
||||
val c = Actor.actorOf[ConsumerActor2]
|
||||
val c = Actor.actorOf(Props[ConsumerActor2]
|
||||
val event = ConsumerActorRegistered.eventFor(c)
|
||||
assert(event === Some(ConsumerActorRegistered(c, consumerOf(c))))
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldCreateNoneFromConsumer = {
|
||||
val event = ConsumerActorRegistered.eventFor(Actor.actorOf[PlainActor])
|
||||
val event = ConsumerActorRegistered.eventFor(Actor.actorOf(Props[PlainActor])
|
||||
assert(event === None)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
service = CamelServiceFactory.createCamelService
|
||||
// register test consumer before registering the publish requestor
|
||||
// and before starting the CamelService (registry is scanned for consumers)
|
||||
actorOf(new TestConsumer("direct:publish-test-1"))
|
||||
actorOf(Props(new TestConsumer("direct:publish-test-1"))
|
||||
service.registerPublishRequestor
|
||||
service.awaitEndpointActivation(1) {
|
||||
service.start
|
||||
|
|
@ -54,7 +54,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
"started" must {
|
||||
"support an in-out message exchange via its endpoint" in {
|
||||
service.awaitEndpointActivation(1) {
|
||||
consumer = actorOf(new TestConsumer("direct:publish-test-2"))
|
||||
consumer = actorOf(Props(new TestConsumer("direct:publish-test-2"))
|
||||
} must be(true)
|
||||
mandatoryTemplate.requestBody("direct:publish-test-2", "msg2") must equal("received msg2")
|
||||
}
|
||||
|
|
@ -119,7 +119,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
"activated with a custom error handler" must {
|
||||
"handle thrown exceptions by generating a custom response" in {
|
||||
service.awaitEndpointActivation(1) {
|
||||
actorOf[ErrorHandlingConsumer]
|
||||
actorOf(Props[ErrorHandlingConsumer]
|
||||
} must be(true)
|
||||
mandatoryTemplate.requestBody("direct:error-handler-test", "hello") must equal("error: hello")
|
||||
|
||||
|
|
@ -128,7 +128,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
"activated with a custom redelivery handler" must {
|
||||
"handle thrown exceptions by redelivering the initial message" in {
|
||||
service.awaitEndpointActivation(1) {
|
||||
actorOf[RedeliveringConsumer]
|
||||
actorOf(Props[RedeliveringConsumer]
|
||||
} must be(true)
|
||||
mandatoryTemplate.requestBody("direct:redelivery-test", "hello") must equal("accepted: hello")
|
||||
|
||||
|
|
@ -143,7 +143,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
var consumer: ActorRef = null
|
||||
|
||||
service.awaitEndpointActivation(1) {
|
||||
consumer = actorOf(new TestAckConsumer("direct:system-ack-test"))
|
||||
consumer = actorOf(Props(new TestAckConsumer("direct:system-ack-test"))
|
||||
} must be(true)
|
||||
|
||||
val endpoint = mandatoryContext.getEndpoint("direct:system-ack-test", classOf[DirectEndpoint])
|
||||
|
|
@ -169,19 +169,19 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
|
||||
"A supervised consumer" must {
|
||||
"be able to reply during receive" in {
|
||||
val consumer = Actor.actorOf(new SupervisedConsumer("reply-channel-test-1"))
|
||||
val consumer = Actor.actorOf(Props(new SupervisedConsumer("reply-channel-test-1"))
|
||||
(consumer ? "succeed").get must equal("ok")
|
||||
}
|
||||
|
||||
"be able to reply on failure during preRestart" in {
|
||||
val consumer = Actor.actorOf(new SupervisedConsumer("reply-channel-test-2"))
|
||||
val consumer = Actor.actorOf(Props(new SupervisedConsumer("reply-channel-test-2"))
|
||||
val supervisor = Supervisor(
|
||||
SupervisorConfig(
|
||||
OneForOneStrategy(List(classOf[Exception]), 2, 10000),
|
||||
Supervise(consumer, Permanent) :: Nil))
|
||||
|
||||
val latch = new CountDownLatch(1)
|
||||
val sender = Actor.actorOf(new Sender("pr", latch))
|
||||
val sender = Actor.actorOf(Props(new Sender("pr", latch))
|
||||
|
||||
consumer.!("fail")(Some(sender))
|
||||
latch.await(5, TimeUnit.SECONDS) must be(true)
|
||||
|
|
@ -195,7 +195,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
|
|||
Supervise(consumer, Temporary) :: Nil))
|
||||
|
||||
val latch = new CountDownLatch(1)
|
||||
val sender = Actor.actorOf(new Sender("ps", latch))
|
||||
val sender = Actor.actorOf(Props(new Sender("ps", latch))
|
||||
|
||||
consumer.!("fail")(Some(sender))
|
||||
latch.await(5, TimeUnit.SECONDS) must be(true)
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message and receive normal response") {
|
||||
given("a registered two-way producer")
|
||||
val producer = actorOf(new TestProducer("direct:producer-test-2", true))
|
||||
val producer = actorOf(Props(new TestProducer("direct:producer-test-2", true))
|
||||
|
||||
when("a test message is sent to the producer with ?")
|
||||
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -44,7 +44,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message and receive failure response") {
|
||||
given("a registered two-way producer")
|
||||
val producer = actorOf(new TestProducer("direct:producer-test-2"))
|
||||
val producer = actorOf(Props(new TestProducer("direct:producer-test-2"))
|
||||
|
||||
when("a test message causing an exception is sent to the producer with ?")
|
||||
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -59,7 +59,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message oneway") {
|
||||
given("a registered one-way producer")
|
||||
val producer = actorOf(new TestProducer("direct:producer-test-1", true) with Oneway)
|
||||
val producer = actorOf(Props(new TestProducer("direct:producer-test-1", true) with Oneway)
|
||||
|
||||
when("a test message is sent to the producer with !")
|
||||
mockEndpoint.expectedBodiesReceived("TEST")
|
||||
|
|
@ -71,7 +71,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message twoway without sender reference") {
|
||||
given("a registered two-way producer")
|
||||
val producer = actorOf(new TestProducer("direct:producer-test-1"))
|
||||
val producer = actorOf(Props(new TestProducer("direct:producer-test-1"))
|
||||
|
||||
when("a test message is sent to the producer with !")
|
||||
mockEndpoint.expectedBodiesReceived("test")
|
||||
|
|
@ -86,7 +86,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message and receive normal response") {
|
||||
given("a registered two-way producer")
|
||||
val producer = actorOf(new TestProducer("direct:producer-test-3"))
|
||||
val producer = actorOf(Props(new TestProducer("direct:producer-test-3"))
|
||||
|
||||
when("a test message is sent to the producer with ?")
|
||||
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -98,7 +98,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message and receive failure response") {
|
||||
given("a registered two-way producer")
|
||||
val producer = actorOf(new TestProducer("direct:producer-test-3"))
|
||||
val producer = actorOf(Props(new TestProducer("direct:producer-test-3"))
|
||||
|
||||
when("a test message causing an exception is sent to the producer with ?")
|
||||
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -116,8 +116,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward normal response to a replying target actor and receive response") {
|
||||
given("a registered two-way producer configured with a forward target")
|
||||
val target = actorOf[ReplyingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-2", target))
|
||||
val target = actorOf(Props[ReplyingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-2", target))
|
||||
|
||||
when("a test message is sent to the producer with ?")
|
||||
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -130,8 +130,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward failure response to a replying target actor and receive response") {
|
||||
given("a registered two-way producer configured with a forward target")
|
||||
val target = actorOf[ReplyingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-2", target))
|
||||
val target = actorOf(Props[ReplyingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-2", target))
|
||||
|
||||
when("a test message causing an exception is sent to the producer with ?")
|
||||
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -146,8 +146,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward normal response to a producing target actor and produce response to direct:forward-test-1") {
|
||||
given("a registered one-way producer configured with a forward target")
|
||||
val target = actorOf[ProducingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-2", target))
|
||||
val target = actorOf(Props[ProducingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-2", target))
|
||||
|
||||
when("a test message is sent to the producer with !")
|
||||
mockEndpoint.expectedBodiesReceived("received test")
|
||||
|
|
@ -159,8 +159,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward failure response to a producing target actor and produce response to direct:forward-test-1") {
|
||||
given("a registered one-way producer configured with a forward target")
|
||||
val target = actorOf[ProducingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-2", target))
|
||||
val target = actorOf(Props[ProducingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-2", target))
|
||||
|
||||
when("a test message causing an exception is sent to the producer with !")
|
||||
mockEndpoint.expectedMessageCount(1)
|
||||
|
|
@ -176,8 +176,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward normal response to a replying target actor and receive response") {
|
||||
given("a registered two-way producer configured with a forward target")
|
||||
val target = actorOf[ReplyingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-3", target))
|
||||
val target = actorOf(Props[ReplyingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-3", target))
|
||||
|
||||
when("a test message is sent to the producer with ?")
|
||||
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -190,8 +190,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward failure response to a replying target actor and receive response") {
|
||||
given("a registered two-way producer configured with a forward target")
|
||||
val target = actorOf[ReplyingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-3", target))
|
||||
val target = actorOf(Props[ReplyingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-3", target))
|
||||
|
||||
when("a test message causing an exception is sent to the producer with ?")
|
||||
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
|
||||
|
|
@ -206,8 +206,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward normal response to a producing target actor and produce response to direct:forward-test-1") {
|
||||
given("a registered one-way producer configured with a forward target")
|
||||
val target = actorOf[ProducingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-3", target))
|
||||
val target = actorOf(Props[ProducingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-3", target))
|
||||
|
||||
when("a test message is sent to the producer with !")
|
||||
mockEndpoint.expectedBodiesReceived("received test")
|
||||
|
|
@ -219,8 +219,8 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
|
|||
|
||||
scenario("produce message, forward failure response to a producing target actor and produce response to direct:forward-test-1") {
|
||||
given("a registered one-way producer configured with a forward target")
|
||||
val target = actorOf[ProducingForwardTarget]
|
||||
val producer = actorOf(new TestForwarder("direct:producer-test-3", target))
|
||||
val target = actorOf(Props[ProducingForwardTarget]
|
||||
val producer = actorOf(Props(new TestForwarder("direct:producer-test-3", target))
|
||||
|
||||
when("a test message causing an exception is sent to the producer with !")
|
||||
mockEndpoint.expectedMessageCount(1)
|
||||
|
|
@ -271,7 +271,7 @@ object ProducerFeatureTest {
|
|||
}
|
||||
|
||||
class TestRoute extends RouteBuilder {
|
||||
val responder = actorOf[TestResponder]
|
||||
val responder = actorOf(Props[TestResponder]
|
||||
def configure {
|
||||
from("direct:forward-test-1").to("mock:mock")
|
||||
// for one-way messaging tests
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
|
|||
import CamelContextManager.mandatoryTemplate
|
||||
|
||||
scenario("one-way communication") {
|
||||
val actor = actorOf[Tester1]
|
||||
val actor = actorOf(Props[Tester1]
|
||||
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
mandatoryTemplate.sendBody("actor:uuid:%s" format actor.uuid, "Martin")
|
||||
assert(latch.await(5000, TimeUnit.MILLISECONDS))
|
||||
|
|
@ -42,7 +42,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
|
|||
}
|
||||
|
||||
scenario("two-way communication") {
|
||||
val actor = actorOf[Tester2]
|
||||
val actor = actorOf(Props[Tester2]
|
||||
assert(mandatoryTemplate.requestBody("actor:uuid:%s" format actor.uuid, "Martin") === "Hello Martin")
|
||||
}
|
||||
|
||||
|
|
@ -70,7 +70,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
|
|||
import CamelContextManager.mandatoryTemplate
|
||||
|
||||
scenario("one-way communication") {
|
||||
val actor = actorOf[Tester1]
|
||||
val actor = actorOf(Props[Tester1]
|
||||
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
mandatoryTemplate.sendBody("actor:%s" format actor.address, "Martin")
|
||||
assert(latch.await(5000, TimeUnit.MILLISECONDS))
|
||||
|
|
@ -79,12 +79,12 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
|
|||
}
|
||||
|
||||
scenario("two-way communication") {
|
||||
val actor = actorOf[Tester2]
|
||||
val actor = actorOf(Props[Tester2]
|
||||
assert(mandatoryTemplate.requestBody("actor:%s" format actor.address, "Martin") === "Hello Martin")
|
||||
}
|
||||
|
||||
scenario("two-way communication via a custom route") {
|
||||
val actor = actorOf[CustomIdActor]("custom-id")
|
||||
val actor = actorOf(Props[CustomIdActor]("custom-id")
|
||||
assert(mandatoryTemplate.requestBody("direct:custom-id-test-1", "Martin") === "Received Martin")
|
||||
assert(mandatoryTemplate.requestBody("direct:custom-id-test-2", "Martin") === "Received Martin")
|
||||
}
|
||||
|
|
@ -113,8 +113,8 @@ object ActorComponentFeatureTest {
|
|||
}
|
||||
|
||||
class TestRoute extends RouteBuilder {
|
||||
val failWithMessage = actorOf[FailWithMessage]
|
||||
val failWithException = actorOf[FailWithException]
|
||||
val failWithMessage = actorOf(Props[FailWithMessage]
|
||||
val failWithException = actorOf(Props[FailWithException]
|
||||
def configure {
|
||||
from("direct:custom-id-test-1").to("actor:custom-id")
|
||||
from("direct:custom-id-test-2").to("actor:id:custom-id")
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldSendMessageToActorWithSyncProcessor = {
|
||||
val actor = actorOf[Tester1]
|
||||
val actor = actorOf(Props[Tester1]
|
||||
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
|
||||
val exchange = endpoint.createExchange(ExchangePattern.InOnly)
|
||||
|
|
@ -38,7 +38,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldSendMessageToActorWithAsyncProcessor = {
|
||||
val actor = actorOf[Tester1]
|
||||
val actor = actorOf(Props[Tester1]
|
||||
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
|
||||
val exchange = endpoint.createExchange(ExchangePattern.InOnly)
|
||||
|
|
@ -53,7 +53,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldSendMessageToActorAndReceiveResponseWithSyncProcessor = {
|
||||
val actor = actorOf(new Tester2 {
|
||||
val actor = actorOf(Props(new Tester2 {
|
||||
override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
|
||||
})
|
||||
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
|
||||
|
|
@ -67,7 +67,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldSendMessageToActorAndReceiveResponseWithAsyncProcessor = {
|
||||
val actor = actorOf(new Tester2 {
|
||||
val actor = actorOf(Props(new Tester2 {
|
||||
override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
|
||||
})
|
||||
val completion = expectAsyncCompletion
|
||||
|
|
@ -83,7 +83,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldSendMessageToActorAndReceiveFailureWithAsyncProcessor = {
|
||||
val actor = actorOf(new Tester2 {
|
||||
val actor = actorOf(Props(new Tester2 {
|
||||
override def response(msg: Message) = Failure(new Exception("testmsg"), Map("k3" -> "v3"))
|
||||
})
|
||||
val completion = expectAsyncCompletion
|
||||
|
|
@ -100,7 +100,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldSendMessageToActorAndReceiveAckWithAsyncProcessor = {
|
||||
val actor = actorOf(new Tester2 {
|
||||
val actor = actorOf(Props(new Tester2 {
|
||||
override def response(msg: Message) = akka.camel.Ack
|
||||
})
|
||||
val completion = expectAsyncCompletion
|
||||
|
|
@ -115,8 +115,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldDynamicallyRouteMessageToActorWithDefaultId = {
|
||||
val actor1 = actorOf[Tester1]("x")
|
||||
val actor2 = actorOf[Tester1]("y")
|
||||
val actor1 = actorOf(Props[Tester1]("x")
|
||||
val actor2 = actorOf(Props[Tester1]("y")
|
||||
actor1
|
||||
actor2
|
||||
val latch1 = (actor1 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
|
|
@ -139,8 +139,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldDynamicallyRouteMessageToActorWithoutDefaultId = {
|
||||
val actor1 = actorOf[Tester1]("x")
|
||||
val actor2 = actorOf[Tester1]("y")
|
||||
val actor1 = actorOf(Props[Tester1]("x")
|
||||
val actor2 = actorOf(Props[Tester1]("y")
|
||||
actor1
|
||||
actor2
|
||||
val latch1 = (actor1 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
|
|
@ -164,8 +164,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldDynamicallyRouteMessageToActorWithDefaultUuid = {
|
||||
val actor1 = actorOf[Tester1]
|
||||
val actor2 = actorOf[Tester1]
|
||||
val actor1 = actorOf(Props[Tester1]
|
||||
val actor2 = actorOf(Props[Tester1]
|
||||
val latch1 = (actor1 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val latch2 = (actor2 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val endpoint = actorEndpoint("actor:uuid:%s" format actor1.uuid)
|
||||
|
|
@ -186,8 +186,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldDynamicallyRouteMessageToActorWithoutDefaultUuid = {
|
||||
val actor1 = actorOf[Tester1]
|
||||
val actor2 = actorOf[Tester1]
|
||||
val actor1 = actorOf(Props[Tester1]
|
||||
val actor2 = actorOf(Props[Tester1]
|
||||
val latch1 = (actor1 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val latch2 = (actor2 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val endpoint = actorEndpoint("actor:uuid:")
|
||||
|
|
@ -209,7 +209,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldThrowExceptionWhenIdNotSet{
|
||||
val actor = actorOf[Tester1]
|
||||
val actor = actorOf(Props[Tester1]
|
||||
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val endpoint = actorEndpoint("actor:id:")
|
||||
intercept[ActorIdentifierNotSetException] {
|
||||
|
|
@ -219,7 +219,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
|
|||
|
||||
@Test
|
||||
def shouldThrowExceptionWhenUuidNotSet{
|
||||
val actor = actorOf[Tester1]
|
||||
val actor = actorOf(Props[Tester1]
|
||||
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
|
||||
val endpoint = actorEndpoint("actor:uuid:")
|
||||
intercept[ActorIdentifierNotSetException] {
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode {
|
|||
Cluster.node.start()
|
||||
barrier("awaitStarted", NrOfNodes).await()
|
||||
|
||||
val ref = Actor.actorOf[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test")
|
||||
val ref = Actor.actorOf(Props[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test")
|
||||
|
||||
ref.isInstanceOf[ClusterActorRef] must be(true)
|
||||
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@
|
|||
// }
|
||||
|
||||
// barrier("create-actor-on-node1", NrOfNodes) {
|
||||
// val actorRef = Actor.actorOf[HelloWorld]("hello-world-write-behind-nosnapshot")
|
||||
// val actorRef = Actor.actorOf(Props[HelloWorld]("hello-world-write-behind-nosnapshot")
|
||||
// // node.isInUseOnNode("hello-world") must be(true)
|
||||
// actorRef.address must be("hello-world-write-behind-nosnapshot")
|
||||
// for (i ← 0 until 10) {
|
||||
|
|
|
|||
|
|
@ -44,7 +44,7 @@
|
|||
// }
|
||||
|
||||
// barrier("create-actor-on-node1", NrOfNodes) {
|
||||
// val actorRef = Actor.actorOf[HelloWorld]("hello-world-write-behind-snapshot")
|
||||
// val actorRef = Actor.actorOf(Props[HelloWorld]("hello-world-write-behind-snapshot")
|
||||
// node.isInUseOnNode("hello-world-write-behind-snapshot") must be(true)
|
||||
// actorRef.address must be("hello-world-write-behind-snapshot")
|
||||
// var counter = 0
|
||||
|
|
|
|||
|
|
@ -44,7 +44,7 @@
|
|||
// }
|
||||
|
||||
// barrier("create-actor-on-node1", NrOfNodes) {
|
||||
// val actorRef = Actor.actorOf[HelloWorld]("hello-world-write-through-nosnapshot")
|
||||
// val actorRef = Actor.actorOf(Props[HelloWorld]("hello-world-write-through-nosnapshot")
|
||||
// actorRef.address must be("hello-world-write-through-nosnapshot")
|
||||
// for (i ← 0 until 10)
|
||||
// (actorRef ? Count(i)).as[String] must be(Some("World from node [node1]"))
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@
|
|||
// }
|
||||
|
||||
// barrier("create-actor-on-node1", NrOfNodes) {
|
||||
// val actorRef = Actor.actorOf[HelloWorld]("hello-world-write-through-snapshot")
|
||||
// val actorRef = Actor.actorOf(Props[HelloWorld]("hello-world-write-through-snapshot")
|
||||
// node.isInUseOnNode("hello-world-write-through-snapshot") must be(true)
|
||||
// actorRef.address must be("hello-world-write-through-snapshot")
|
||||
// var counter = 0
|
||||
|
|
|
|||
|
|
@ -44,7 +44,7 @@ class DirectRoutingFailoverMultiJvmNode1 extends MasterClusterTestNode {
|
|||
}
|
||||
|
||||
LocalCluster.barrier("actor-creation", NrOfNodes) {
|
||||
actor = Actor.actorOf[SomeActor]("service-hello")
|
||||
actor = Actor.actorOf(Props[SomeActor]("service-hello")
|
||||
}
|
||||
|
||||
LocalCluster.barrier("verify-actor", NrOfNodes) {
|
||||
|
|
|
|||
|
|
@ -45,11 +45,11 @@ class HomeNodeMultiJvmNode2 extends ClusterTestNode {
|
|||
Cluster.node.start()
|
||||
barrier("waiting-for-begin", NrOfNodes).await()
|
||||
|
||||
val actorNode1 = Actor.actorOf[SomeActor]("service-node1")
|
||||
val actorNode1 = Actor.actorOf(Props[SomeActor]("service-node1")
|
||||
val name1 = (actorNode1 ? "identify").get.asInstanceOf[String]
|
||||
name1 must equal("node1")
|
||||
|
||||
val actorNode2 = Actor.actorOf[SomeActor]("service-node2")
|
||||
val actorNode2 = Actor.actorOf(Props[SomeActor]("service-node2")
|
||||
val name2 = (actorNode2 ? "identify").get.asInstanceOf[String]
|
||||
name2 must equal("node2")
|
||||
|
||||
|
|
|
|||
|
|
@ -48,7 +48,7 @@ class SingleReplicaDirectRoutingMultiJvmNode2 extends ClusterTestNode {
|
|||
Cluster.node.start()
|
||||
LocalCluster.barrier("waiting-for-begin", NrOfNodes).await()
|
||||
|
||||
val actor = Actor.actorOf[SomeActor]("service-hello").asInstanceOf[ClusterActorRef]
|
||||
val actor = Actor.actorOf(Props[SomeActor]("service-hello").asInstanceOf[ClusterActorRef]
|
||||
actor.isRunning must be(true)
|
||||
|
||||
val result = (actor ? "identify").get
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ class RandomFailoverMultiJvmNode1 extends MasterClusterTestNode {
|
|||
}
|
||||
|
||||
barrier("actor-creation", NrOfNodes) {
|
||||
actor = Actor.actorOf[SomeActor]("service-hello")
|
||||
actor = Actor.actorOf(Props[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[ClusterActorRef] must be(true)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -45,11 +45,11 @@ class HomeNodeMultiJvmNode2 extends ClusterTestNode {
|
|||
Cluster.node.start()
|
||||
barrier("waiting-for-begin", NrOfNodes).await()
|
||||
|
||||
val actorNode1 = Actor.actorOf[SomeActor]("service-node1")
|
||||
val actorNode1 = Actor.actorOf(Props[SomeActor]("service-node1")
|
||||
val nameNode1 = (actorNode1 ? "identify").get.asInstanceOf[String]
|
||||
nameNode1 must equal("node1")
|
||||
|
||||
val actorNode2 = Actor.actorOf[SomeActor]("service-node2")
|
||||
val actorNode2 = Actor.actorOf(Props[SomeActor]("service-node2")
|
||||
val nameNode2 = (actorNode2 ? "identify").get.asInstanceOf[String]
|
||||
nameNode2 must equal("node2")
|
||||
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ class Random1ReplicaMultiJvmNode1 extends MasterClusterTestNode {
|
|||
"create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in {
|
||||
Cluster.node.start()
|
||||
|
||||
var hello = Actor.actorOf[HelloWorld]("service-hello")
|
||||
var hello = Actor.actorOf(Props[HelloWorld]("service-hello")
|
||||
hello must not equal (null)
|
||||
hello.address must equal("service-hello")
|
||||
hello.isInstanceOf[ClusterActorRef] must be(true)
|
||||
|
|
|
|||
|
|
@ -64,7 +64,7 @@ class Random3ReplicasMultiJvmNode2 extends ClusterTestNode {
|
|||
|
||||
//check if the actorRef is the expected remoteActorRef.
|
||||
var hello: ActorRef = null
|
||||
hello = Actor.actorOf[HelloWorld]("service-hello")
|
||||
hello = Actor.actorOf(Props[HelloWorld]("service-hello")
|
||||
hello must not equal (null)
|
||||
hello.address must equal("service-hello")
|
||||
hello.isInstanceOf[ClusterActorRef] must be(true)
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ class RoundRobinFailoverMultiJvmNode1 extends MasterClusterTestNode {
|
|||
}
|
||||
|
||||
barrier("actor-creation", NrOfNodes) {
|
||||
actor = Actor.actorOf[SomeActor]("service-hello")
|
||||
actor = Actor.actorOf(Props[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[ClusterActorRef] must be(true)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -48,11 +48,11 @@ class HomeNodeMultiJvmNode2 extends ClusterTestNode {
|
|||
Cluster.node.start()
|
||||
barrier("waiting-for-begin", NrOfNodes).await()
|
||||
|
||||
val actorNode1 = Actor.actorOf[SomeActor]("service-node1")
|
||||
val actorNode1 = Actor.actorOf(Props[SomeActor]("service-node1")
|
||||
val name1 = (actorNode1 ? "identify").get.asInstanceOf[String]
|
||||
name1 must equal("node1")
|
||||
|
||||
val actorNode2 = Actor.actorOf[SomeActor]("service-node2")
|
||||
val actorNode2 = Actor.actorOf(Props[SomeActor]("service-node2")
|
||||
val name2 = (actorNode2 ? "identify").get.asInstanceOf[String]
|
||||
name2 must equal("node2")
|
||||
|
||||
|
|
|
|||
|
|
@ -35,7 +35,7 @@ class RoundRobin1ReplicaMultiJvmNode1 extends MasterClusterTestNode {
|
|||
"create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in {
|
||||
Cluster.node.start()
|
||||
|
||||
var hello = Actor.actorOf[HelloWorld]("service-hello")
|
||||
var hello = Actor.actorOf(Props[HelloWorld]("service-hello")
|
||||
hello must not equal (null)
|
||||
hello.address must equal("service-hello")
|
||||
hello.isInstanceOf[ClusterActorRef] must be(true)
|
||||
|
|
|
|||
|
|
@ -89,7 +89,7 @@ class RoundRobin2ReplicasMultiJvmNode2 extends ClusterTestNode {
|
|||
//check if the actorRef is the expected remoteActorRef.
|
||||
var hello: ActorRef = null
|
||||
barrier("get-ref-to-actor-on-node2", NrOfNodes) {
|
||||
hello = Actor.actorOf[HelloWorld]("service-hello")
|
||||
hello = Actor.actorOf(Props[HelloWorld]("service-hello")
|
||||
hello must not equal (null)
|
||||
hello.address must equal("service-hello")
|
||||
hello.isInstanceOf[ClusterActorRef] must be(true)
|
||||
|
|
|
|||
|
|
@ -91,7 +91,7 @@
|
|||
// //check if the actorRef is the expected remoteActorRef.
|
||||
// var hello: ActorRef = null
|
||||
// barrier("get-ref-to-actor-on-node2", NrOfNodes) {
|
||||
// hello = Actor.actorOf[HelloWorld]("service-hello")
|
||||
// hello = Actor.actorOf(Props[HelloWorld]("service-hello")
|
||||
// hello must not equal (null)
|
||||
// hello.address must equal("service-hello")
|
||||
// hello.isInstanceOf[ClusterActorRef] must be(true)
|
||||
|
|
|
|||
|
|
@ -63,7 +63,7 @@ class ScatterGatherFailoverMultiJvmNode1 extends MasterClusterTestNode {
|
|||
/*
|
||||
FIXME: Uncomment, when custom routers will be fully supported (ticket #1109)
|
||||
|
||||
val actor = Actor.actorOf[TestActor]("service-hello").asInstanceOf[ClusterActorRef]
|
||||
val actor = Actor.actorOf(Props[TestActor]("service-hello").asInstanceOf[ClusterActorRef]
|
||||
|
||||
identifyConnections(actor).size() must be(2)
|
||||
|
||||
|
|
|
|||
|
|
@ -48,7 +48,7 @@ cluster node.
|
|||
Cluster configuration
|
||||
~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
Cluster is configured in the ``akka.cloud.cluster`` section in the :ref:`configuration`.
|
||||
Cluster is configured in the ``akka.cloud.cluster`` section in the :ref:`configuration`.
|
||||
Here you specify the default addresses to the ZooKeeper
|
||||
servers, timeouts, if compression should be on or off, and so on.
|
||||
|
||||
|
|
@ -328,7 +328,7 @@ created actor::
|
|||
|
||||
val clusterNode = Cluster.newNode(NodeAddress("test-cluster", "node1")).start
|
||||
|
||||
val hello = actorOf[HelloActor].start.asInstanceOf[LocalActorRef]
|
||||
val hello = actorOf(Props[HelloActor].start.asInstanceOf[LocalActorRef]
|
||||
|
||||
val serializeMailbox = false
|
||||
val nrOfInstances = 5
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ object Pi extends App {
|
|||
|
||||
//#create-workers
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf(Props[Worker])
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
val router = Routing.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ There is an Event Handler which takes the place of a logging system in Akka:
|
|||
|
||||
akka.event.EventHandler
|
||||
|
||||
You can configure which event handlers should be registered at boot time. That is done using the 'event-handlers' element in
|
||||
You can configure which event handlers should be registered at boot time. That is done using the 'event-handlers' element in
|
||||
the :ref:`configuration`. Here you can also define the log level.
|
||||
|
||||
.. code-block:: ruby
|
||||
|
|
@ -26,7 +26,7 @@ Example of creating a listener from Scala (from Java you just have to create an
|
|||
|
||||
.. code-block:: scala
|
||||
|
||||
val errorHandlerEventListener = Actor.actorOf(new Actor {
|
||||
val errorHandlerEventListener = Actor.actorOf(Props(new Actor {
|
||||
self.dispatcher = EventHandler.EventHandlerDispatcher
|
||||
|
||||
def receive = {
|
||||
|
|
|
|||
|
|
@ -66,7 +66,7 @@
|
|||
|
||||
// //#create-workers
|
||||
// // create the workers
|
||||
// val workers = Vector.fill(nrOfWorkers)(system.actorOf[Worker])
|
||||
// val workers = Vector.fill(nrOfWorkers)(system.actorOf(Props[Worker])
|
||||
|
||||
// // wrap them with a load-balancing router
|
||||
// val router = system.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
|
||||
|
|
@ -119,7 +119,7 @@
|
|||
// val latch = new CountDownLatch(1)
|
||||
|
||||
// // create the master
|
||||
// val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
|
||||
// val master = system.actorOf(Props(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
|
||||
|
||||
// // start the calculation
|
||||
// master ! Calculate
|
||||
|
|
|
|||
|
|
@ -158,8 +158,8 @@ Here is the layout that Maven created::
|
|||
|
||||
As you can see we already have a Java source file called ``App.java``, let's now rename it to ``Pi.java``.
|
||||
|
||||
We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. The Akka Maven repository can be found at `<http://akka.io/repository>`_
|
||||
and Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
|
||||
We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. The Akka Maven repository can be found at `<http://akka.io/repository>`_
|
||||
and Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
|
||||
It should now look something like this:
|
||||
|
||||
.. code-block:: xml
|
||||
|
|
@ -341,7 +341,7 @@ The master actor is a little bit more involved. In its constructor we need to cr
|
|||
}
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
ActorRef router = actorOf(new UntypedActorFactory() {
|
||||
ActorRef router = actorOf(Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new PiRouter(workers);
|
||||
}
|
||||
|
|
@ -359,7 +359,7 @@ One thing to note is that we used two different versions of the ``actorOf`` meth
|
|||
|
||||
The actor's life-cycle is:
|
||||
|
||||
- Created & Started -- ``Actor.actorOf[MyActor]`` -- can receive messages
|
||||
- Created & Started -- ``Actor.actorOf(Props[MyActor]`` -- can receive messages
|
||||
- Stopped -- ``actorRef.stop()`` -- can **not** receive messages
|
||||
|
||||
Once the actor has been stopped it is dead and can not be started again.
|
||||
|
|
@ -408,7 +408,7 @@ Here is the master actor::
|
|||
}
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
router = actorOf(new UntypedActorFactory() {
|
||||
router = actorOf(Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new PiRouter(workers);
|
||||
}
|
||||
|
|
@ -495,7 +495,7 @@ Now the only thing that is left to implement is the runner that should bootstrap
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
// create the master
|
||||
ActorRef master = actorOf(new UntypedActorFactory() {
|
||||
ActorRef master = actorOf(Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
|
||||
}
|
||||
|
|
@ -633,7 +633,7 @@ Before we package it up and run it, let's take a look at the full code now, with
|
|||
}
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
router = actorOf(new UntypedActorFactory() {
|
||||
router = actorOf(Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new PiRouter(workers);
|
||||
}
|
||||
|
|
@ -691,7 +691,7 @@ Before we package it up and run it, let's take a look at the full code now, with
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
// create the master
|
||||
ActorRef master = actorOf(new UntypedActorFactory() {
|
||||
ActorRef master = actorOf(Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -157,7 +157,7 @@ If you have not already done so, now is the time to create an Eclipse project fo
|
|||
Using SBT in Eclipse
|
||||
^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
If you are an `SBT <https://github.com/harrah/xsbt/wiki>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbteclipse`` plugin. This adds support for generating Eclipse project files from your SBT project.
|
||||
If you are an `SBT <https://github.com/harrah/xsbt/wiki>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbteclipse`` plugin. This adds support for generating Eclipse project files from your SBT project.
|
||||
You need to install the plugin as described in the `README of sbteclipse <https://github.com/typesafehub/sbteclipse>`_
|
||||
|
||||
Then run the ``eclipse`` target to generate the Eclipse project::
|
||||
|
|
@ -253,7 +253,7 @@ Now create a new class for the master actor. The master actor is a little bit mo
|
|||
and then we can create the workers::
|
||||
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf(Props[Worker])
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
val router = Routing.loadBalancerActor(CyclicIterator(workers))
|
||||
|
|
@ -262,11 +262,11 @@ As you can see we are using the ``actorOf`` factory method to create actors, thi
|
|||
|
||||
import akka.actor.Actor.actorOf
|
||||
|
||||
There are two versions of ``actorOf``; one of them taking a actor type and the other one an instance of an actor. The former one (``actorOf[MyActor]``) is used when the actor class has a no-argument constructor while the second one (``actorOf(new MyActor(..))``) is used when the actor class has a constructor that takes arguments. This is the only way to create an instance of an Actor and the ``actorOf`` method ensures this. The latter version is using call-by-name and lazily creates the actor within the scope of the ``actorOf`` method. The ``actorOf`` method instantiates the actor and returns, not an instance to the actor, but an instance to an ``ActorRef``. This reference is the handle through which you communicate with the actor. It is immutable, serializable and location-aware meaning that it "remembers" its original actor even if it is sent to other nodes across the network and can be seen as the equivalent to the Erlang actor's PID.
|
||||
There are two versions of ``actorOf``; one of them taking a actor type and the other one an instance of an actor. The former one (``actorOf(Props[MyActor]``) is used when the actor class has a no-argument constructor while the second one (``actorOf(Props(new MyActor(..))``) is used when the actor class has a constructor that takes arguments. This is the only way to create an instance of an Actor and the ``actorOf`` method ensures this. The latter version is using call-by-name and lazily creates the actor within the scope of the ``actorOf`` method. The ``actorOf`` method instantiates the actor and returns, not an instance to the actor, but an instance to an ``ActorRef``. This reference is the handle through which you communicate with the actor. It is immutable, serializable and location-aware meaning that it "remembers" its original actor even if it is sent to other nodes across the network and can be seen as the equivalent to the Erlang actor's PID.
|
||||
|
||||
The actor's life-cycle is:
|
||||
|
||||
- Created -- ``Actor.actorOf[MyActor]`` -- can **not** receive messages
|
||||
- Created -- ``Actor.actorOf(Props[MyActor]`` -- can **not** receive messages
|
||||
- Started -- ``actorRef`` -- can receive messages
|
||||
- Stopped -- ``actorRef.stop()`` -- can **not** receive messages
|
||||
|
||||
|
|
@ -289,7 +289,7 @@ Here is the master actor::
|
|||
var start: Long = _
|
||||
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
|
||||
val workers = Vector.fill(nrOfWorkers)(actorOf(Props[Worker])
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
val router = Routing.loadBalancerActor(CyclicIterator(workers))
|
||||
|
|
|
|||
|
|
@ -303,9 +303,9 @@ imported::
|
|||
import akka.actor.Actor.actorOf
|
||||
|
||||
There are two versions of ``actorOf``; one of them taking a actor type and the
|
||||
other one an instance of an actor. The former one (``actorOf[MyActor]``) is used
|
||||
other one an instance of an actor. The former one (``actorOf(Props[MyActor]``) is used
|
||||
when the actor class has a no-argument constructor while the second one
|
||||
(``actorOf(new MyActor(..))``) is used when the actor class has a constructor
|
||||
(``actorOf(Props(new MyActor(..))``) is used when the actor class has a constructor
|
||||
that takes arguments. This is the only way to create an instance of an Actor and
|
||||
the ``actorOf`` method ensures this. The latter version is using call-by-name
|
||||
and lazily creates the actor within the scope of the ``actorOf`` method. The
|
||||
|
|
@ -318,7 +318,7 @@ Erlang actor's PID.
|
|||
|
||||
The actor's life-cycle is:
|
||||
|
||||
- Created & Started -- ``Actor.actorOf[MyActor]`` -- can receive messages
|
||||
- Created & Started -- ``Actor.actorOf(Props[MyActor]`` -- can receive messages
|
||||
- Stopped -- ``actorRef.stop()`` -- can **not** receive messages
|
||||
|
||||
Once the actor has been stopped it is dead and can not be started again.
|
||||
|
|
|
|||
|
|
@ -1,12 +1,13 @@
|
|||
package akka.docs.actor;
|
||||
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import static akka.actor.Actors.*;
|
||||
import akka.actor.UntypedActor;
|
||||
|
||||
//#context-actorOf
|
||||
public class FirstUntypedActor extends UntypedActor {
|
||||
ActorRef myActor = getContext().actorOf(MyActor.class);
|
||||
ActorRef myActor = getContext().actorOf(new Props(MyActor.class));
|
||||
|
||||
//#context-actorOf
|
||||
|
||||
|
|
|
|||
|
|
@ -2,6 +2,7 @@ package akka.docs.actor;
|
|||
|
||||
import static akka.docs.actor.UntypedActorSwapper.Swap.SWAP;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.UnhandledMessageException;
|
||||
import akka.actor.UntypedActor;
|
||||
|
|
@ -40,7 +41,7 @@ public class UntypedActorSwapper {
|
|||
|
||||
public static void main(String... args) {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef swap = system.actorOf(Swapper.class);
|
||||
ActorRef swap = system.actorOf(new Props(Swapper.class));
|
||||
swap.tell(SWAP); // logs Hi
|
||||
swap.tell(SWAP); // logs Ho
|
||||
swap.tell(SWAP); // logs Hi
|
||||
|
|
@ -50,4 +51,4 @@ public class UntypedActorSwapper {
|
|||
}
|
||||
|
||||
}
|
||||
//#swapper
|
||||
//#swapper
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@ package akka.docs.actor;
|
|||
//#imports
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
|
||||
//#imports
|
||||
|
||||
|
|
@ -38,7 +39,7 @@ public class UntypedActorTestBase {
|
|||
public void systemActorOf() {
|
||||
//#system-actorOf
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef myActor = system.actorOf(MyUntypedActor.class);
|
||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class));
|
||||
//#system-actorOf
|
||||
myActor.tell("test");
|
||||
system.stop();
|
||||
|
|
@ -48,7 +49,7 @@ public class UntypedActorTestBase {
|
|||
public void contextActorOf() {
|
||||
//#context-actorOf
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef myActor = system.actorOf(MyUntypedActor.class);
|
||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class));
|
||||
//#context-actorOf
|
||||
myActor.tell("test");
|
||||
system.stop();
|
||||
|
|
@ -59,11 +60,11 @@ public class UntypedActorTestBase {
|
|||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
//#creating-constructor
|
||||
// allows passing in arguments to the MyActor constructor
|
||||
ActorRef myActor = system.actorOf(new UntypedActorFactory() {
|
||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new MyActor("...");
|
||||
}
|
||||
});
|
||||
}));
|
||||
//#creating-constructor
|
||||
myActor.tell("test");
|
||||
system.stop();
|
||||
|
|
@ -74,8 +75,7 @@ public class UntypedActorTestBase {
|
|||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
//#creating-props
|
||||
MessageDispatcher dispatcher = system.dispatcherFactory().newFromConfig("my-dispatcher");
|
||||
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher),
|
||||
"myactor");
|
||||
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher), "myactor");
|
||||
//#creating-props
|
||||
myActor.tell("test");
|
||||
system.stop();
|
||||
|
|
@ -84,11 +84,11 @@ public class UntypedActorTestBase {
|
|||
@Test
|
||||
public void usingAsk() {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef myActor = system.actorOf(new UntypedActorFactory() {
|
||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new MyAskActor();
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
//#using-ask
|
||||
Future future = myActor.ask("Hello", 1000);
|
||||
|
|
@ -109,7 +109,7 @@ public class UntypedActorTestBase {
|
|||
@Test
|
||||
public void receiveTimeout() {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef myActor = system.actorOf(MyReceivedTimeoutUntypedActor.class);
|
||||
ActorRef myActor = system.actorOf(new Props(MyReceivedTimeoutUntypedActor.class));
|
||||
myActor.tell("Hello");
|
||||
system.stop();
|
||||
}
|
||||
|
|
@ -117,7 +117,7 @@ public class UntypedActorTestBase {
|
|||
@Test
|
||||
public void usePoisonPill() {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef myActor = system.actorOf(MyUntypedActor.class);
|
||||
ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class));
|
||||
//#poison-pill
|
||||
myActor.tell(poisonPill());
|
||||
//#poison-pill
|
||||
|
|
@ -127,7 +127,7 @@ public class UntypedActorTestBase {
|
|||
@Test
|
||||
public void useKill() {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef victim = system.actorOf(MyUntypedActor.class);
|
||||
ActorRef victim = system.actorOf(new Props(MyUntypedActor.class));
|
||||
//#kill
|
||||
victim.tell(kill());
|
||||
//#kill
|
||||
|
|
@ -137,11 +137,11 @@ public class UntypedActorTestBase {
|
|||
@Test
|
||||
public void useBecome() {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
ActorRef myActor = system.actorOf(new UntypedActorFactory() {
|
||||
ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new HotSwapActor();
|
||||
}
|
||||
});
|
||||
}));
|
||||
myActor.tell("foo");
|
||||
myActor.tell("bar");
|
||||
myActor.tell("bar");
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ one-liner. Here's an example.
|
|||
}
|
||||
|
||||
// start and expose actor via tcp
|
||||
val myActor = actorOf[MyActor]
|
||||
val myActor = actorOf(Props[MyActor]
|
||||
|
||||
The above example exposes an actor over a tcp endpoint on port 6200 via Apache
|
||||
Camel's `Mina component`_. The actor implements the endpointUri method to define
|
||||
|
|
@ -362,7 +362,7 @@ after the ActorRef method returned.
|
|||
|
||||
import akka.actor.Actor._
|
||||
|
||||
val actor = actorOf[Consumer1] // create Consumer actor and activate endpoint in background
|
||||
val actor = actorOf(Props[Consumer1] // create Consumer actor and activate endpoint in background
|
||||
|
||||
**Java**
|
||||
|
||||
|
|
@ -544,7 +544,7 @@ still in progress after the ``ActorRef.stop`` method returned.
|
|||
|
||||
import akka.actor.Actor._
|
||||
|
||||
val actor = actorOf[Consumer1] // create Consumer actor
|
||||
val actor = actorOf(Props[Consumer1] // create Consumer actor
|
||||
actor // activate endpoint in background
|
||||
// ...
|
||||
actor.stop // deactivate endpoint in background
|
||||
|
|
@ -872,7 +872,7 @@ actor and register it at the remote server.
|
|||
// ...
|
||||
startCamelService
|
||||
|
||||
val consumer = val consumer = actorOf[RemoteActor1]
|
||||
val consumer = val consumer = actorOf(Props[RemoteActor1]
|
||||
|
||||
remote.start("localhost", 7777)
|
||||
remote.register(consumer) // register and start remote consumer
|
||||
|
|
@ -1028,7 +1028,7 @@ used.
|
|||
import akka.actor.Actor._
|
||||
import akka.actor.ActorRef
|
||||
|
||||
val producer = actorOf[Producer1]
|
||||
val producer = actorOf(Props[Producer1]
|
||||
val response = (producer ? "akka rocks").get
|
||||
val body = response.bodyAs[String]
|
||||
|
||||
|
|
@ -1156,7 +1156,7 @@ argument.
|
|||
import akka.actor.ActorRef;
|
||||
|
||||
ActorRef target = ...
|
||||
ActorRef producer = actorOf(new Producer1Factory(target));
|
||||
ActorRef producer = actorOf(Props(new Producer1Factory(target));
|
||||
producer;
|
||||
|
||||
Before producing messages to endpoints, producer actors can pre-process them by
|
||||
|
|
@ -1946,7 +1946,7 @@ ends at the target actor.
|
|||
import akka.camel.{Message, CamelContextManager, CamelServiceManager}
|
||||
|
||||
object CustomRouteExample extends Application {
|
||||
val target = actorOf[CustomRouteTarget]
|
||||
val target = actorOf(Props[CustomRouteTarget]
|
||||
|
||||
CamelServiceManager.startCamelService
|
||||
CamelContextManager.mandatoryContext.addRoutes(new CustomRouteBuilder(target.uuid))
|
||||
|
|
@ -2545,9 +2545,9 @@ as shown in the following snippet (see also `sample.camel.Boot`_).
|
|||
}
|
||||
|
||||
// Wire and start the example actors
|
||||
val httpTransformer = actorOf(new HttpTransformer)
|
||||
val httpProducer = actorOf(new HttpProducer(httpTransformer))
|
||||
val httpConsumer = actorOf(new HttpConsumer(httpProducer))
|
||||
val httpTransformer = actorOf(Props(new HttpTransformer)
|
||||
val httpProducer = actorOf(Props(new HttpProducer(httpTransformer))
|
||||
val httpConsumer = actorOf(Props(new HttpConsumer(httpProducer))
|
||||
|
||||
The `jetty endpoints`_ of HttpConsumer and HttpProducer support asynchronous
|
||||
in-out message exchanges and do not allocate threads for the full duration of
|
||||
|
|
@ -2637,9 +2637,9 @@ follows.
|
|||
CamelContextManager.init()
|
||||
CamelContextManager.mandatoryContext.addRoutes(new CustomRouteBuilder)
|
||||
|
||||
val producer = actorOf[Producer1]
|
||||
val mediator = actorOf(new Transformer(producer))
|
||||
val consumer = actorOf(new Consumer3(mediator))
|
||||
val producer = actorOf(Props[Producer1]
|
||||
val mediator = actorOf(Props(new Transformer(producer))
|
||||
val consumer = actorOf(Props(new Consumer3(mediator))
|
||||
}
|
||||
|
||||
class CustomRouteBuilder extends RouteBuilder {
|
||||
|
|
@ -2741,11 +2741,11 @@ Wiring these actors to implement the above example is as simple as
|
|||
|
||||
// Setup publish/subscribe example
|
||||
val jmsUri = "jms:topic:test"
|
||||
val jmsSubscriber1 = actorOf(new Subscriber("jms-subscriber-1", jmsUri))
|
||||
val jmsSubscriber2 = actorOf(new Subscriber("jms-subscriber-2", jmsUri))
|
||||
val jmsPublisher = actorOf(new Publisher("jms-publisher", jmsUri))
|
||||
val jmsSubscriber1 = actorOf(Props(new Subscriber("jms-subscriber-1", jmsUri))
|
||||
val jmsSubscriber2 = actorOf(Props(new Subscriber("jms-subscriber-2", jmsUri))
|
||||
val jmsPublisher = actorOf(Props(new Publisher("jms-publisher", jmsUri))
|
||||
|
||||
val jmsPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/jms", jmsPublisher))
|
||||
val jmsPublisherBridge = actorOf(Props(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/jms", jmsPublisher))
|
||||
}
|
||||
|
||||
To publish messages to subscribers one could of course also use the JMS API
|
||||
|
|
@ -2838,10 +2838,10 @@ to be changed.
|
|||
|
||||
// Setup publish/subscribe example
|
||||
val cometdUri = "cometd://localhost:8111/test/abc?resourceBase=target"
|
||||
val cometdSubscriber = actorOf(new Subscriber("cometd-subscriber", cometdUri))
|
||||
val cometdPublisher = actorOf(new Publisher("cometd-publisher", cometdUri))
|
||||
val cometdSubscriber = actorOf(Props(new Subscriber("cometd-subscriber", cometdUri))
|
||||
val cometdPublisher = actorOf(Props(new Publisher("cometd-publisher", cometdUri))
|
||||
|
||||
val cometdPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/cometd", cometdPublisher))
|
||||
val cometdPublisherBridge = actorOf(Props(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/cometd", cometdPublisher))
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -2884,7 +2884,7 @@ seconds:
|
|||
startCamelService
|
||||
|
||||
// create and start a quartz actor
|
||||
val myActor = actorOf[MyQuartzActor]
|
||||
val myActor = actorOf(Props[MyQuartzActor]
|
||||
|
||||
} // end main
|
||||
|
||||
|
|
|
|||
|
|
@ -133,7 +133,7 @@ Important changes from RC2-RC3
|
|||
|
||||
**classes RemoteActor, RemoteUntypedActor and RemoteUntypedConsumerActors has been deleted and replaced with akka.actor.Actors.remote().actorOf(x, host port)/akka.actor.Actor.remote.actorOf(x, host, port)**
|
||||
|
||||
- RemoteActor, RemoteUntypedActor - deleted, use: remote().actorOf(YourActor.class, host, port) (Java) or remote.actorOf[YourActor](host, port)
|
||||
- RemoteActor, RemoteUntypedActor - deleted, use: remote().actorOf(YourActor.class, host, port) (Java) or remote.actorOf(Props[YourActor](host, port)
|
||||
|
||||
**Remoted spring-actors now default to spring id as service-name, use "service-name" attribute on "remote"-tag to override**
|
||||
|
||||
|
|
@ -369,7 +369,7 @@ The API for waiting for consumer endpoint activation and de-activation has been
|
|||
import se.scalablesolutions.akka.camel.CamelServiceManager._
|
||||
|
||||
val s = startCamelService
|
||||
val actor = Actor.actorOf[SampleConsumer]
|
||||
val actor = Actor.actorOf(Props[SampleConsumer]
|
||||
|
||||
// wait for 1 consumer being activated
|
||||
s.awaitEndpointActivation(1) {
|
||||
|
|
|
|||
|
|
@ -35,14 +35,14 @@ To:
|
|||
.. code-block:: scala
|
||||
|
||||
import Actor._
|
||||
val a = actorOf[MyActor]
|
||||
val a = actorOf(Props[MyActor]
|
||||
a ! msg
|
||||
|
||||
You can also start it in the same statement:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val a = actorOf[MyActor]
|
||||
val a = actorOf(Props[MyActor]
|
||||
|
||||
Creating Actors with non-default constructor
|
||||
--------------------------------------------
|
||||
|
|
@ -59,7 +59,7 @@ To:
|
|||
.. code-block:: scala
|
||||
|
||||
import Actor._
|
||||
val a = actorOf(new MyActor(..))
|
||||
val a = actorOf(Props(new MyActor(..))
|
||||
a ! msg
|
||||
|
||||
Use of 'self' ActorRef API
|
||||
|
|
@ -119,7 +119,7 @@ If you want to serialize it yourself, here is how to do it:
|
|||
|
||||
.. code-block:: scala
|
||||
|
||||
val actorRef1 = actorOf[MyActor]
|
||||
val actorRef1 = actorOf(Props[MyActor]
|
||||
|
||||
val bytes = actorRef1.toBinary
|
||||
|
||||
|
|
@ -129,7 +129,7 @@ If you are also using Protobuf then you can use the methods that work with Proto
|
|||
|
||||
.. code-block:: scala
|
||||
|
||||
val actorRef1 = actorOf[MyActor]
|
||||
val actorRef1 = actorOf(Props[MyActor]
|
||||
|
||||
val protobufMessage = actorRef1.toProtocol
|
||||
|
||||
|
|
@ -166,7 +166,7 @@ They will be removed in 1.0. Instead use
|
|||
|
||||
Usage example:
|
||||
.. code-block:: scala
|
||||
|
||||
|
||||
val m = Message(1.4)
|
||||
val b = m.bodyAs[String]
|
||||
|
||||
|
|
|
|||
|
|
@ -88,7 +88,7 @@ So your listener Actor needs to be able to handle these messages. Example:
|
|||
class RegistryListener extends Actor {
|
||||
def receive = {
|
||||
case event: ActorRegistered =>
|
||||
EventHandler.info(this, "Actor registered: %s - %s".format(
|
||||
EventHandler.info(this, "Actor registered: %s - %s".format(
|
||||
event.actor.actorClassName, event.actor.uuid))
|
||||
case event: ActorUnregistered =>
|
||||
// ...
|
||||
|
|
@ -102,5 +102,5 @@ The above actor can be added as listener of registry events:
|
|||
import akka.actor._
|
||||
import akka.actor.Actor._
|
||||
|
||||
val listener = actorOf[RegistryListener]
|
||||
val listener = actorOf(Props[RegistryListener]
|
||||
registry.addListener(listener)
|
||||
|
|
|
|||
|
|
@ -73,7 +73,7 @@ a top level actor, that is supervised by the system (internal guardian actor).
|
|||
.. includecode:: code/ActorDocSpec.scala#context-actorOf
|
||||
|
||||
Actors are automatically started asynchronously when created.
|
||||
When you create the ``Actor`` then it will automatically call the ``preStart``
|
||||
When you create the ``Actor`` then it will automatically call the ``preStart``
|
||||
callback method on the ``Actor`` trait. This is an excellent place to
|
||||
add initialization code for the actor.
|
||||
|
||||
|
|
@ -87,7 +87,7 @@ Creating Actors with non-default constructor
|
|||
--------------------------------------------
|
||||
|
||||
If your Actor has a constructor that takes parameters then you can't create it
|
||||
using ``actorOf[TYPE]``. Instead you can use a variant of ``actorOf`` that takes
|
||||
using ``actorOf(Props[TYPE]``. Instead you can use a variant of ``actorOf`` that takes
|
||||
a call-by-name block in which you can create the Actor in any way you like.
|
||||
|
||||
Here is an example:
|
||||
|
|
@ -98,7 +98,7 @@ Here is an example:
|
|||
Creating Actors with Props
|
||||
--------------------------
|
||||
|
||||
``Props`` is a configuration object to specify additional things for the actor to
|
||||
``Props`` is a configuration object to specify additional things for the actor to
|
||||
be created, such as the ``MessageDispatcher``.
|
||||
|
||||
.. includecode:: code/ActorDocSpec.scala#creating-props
|
||||
|
|
@ -128,7 +128,7 @@ Actor API
|
|||
The :class:`Actor` trait defines only one abstract method, the above mentioned
|
||||
:meth:`receive`, which implements the behavior of the actor.
|
||||
|
||||
If the current actor behavior does not match a received message, :meth:`unhandled`
|
||||
If the current actor behavior does not match a received message, :meth:`unhandled`
|
||||
is called, which by default throws an :class:`UnhandledMessageException`.
|
||||
|
||||
In addition, it offers:
|
||||
|
|
@ -145,7 +145,7 @@ In addition, it offers:
|
|||
|
||||
You can import the members in the :obj:`context` to avoid prefixing access with ``context.``
|
||||
|
||||
.. includecode:: code/ActorDocSpec.scala#import-context
|
||||
.. includecode:: code/ActorDocSpec.scala#import-context
|
||||
|
||||
The remaining visible methods are user-overridable life-cycle hooks which are
|
||||
described in the following::
|
||||
|
|
@ -195,7 +195,7 @@ processing a message. This restart involves the hooks mentioned above:
|
|||
|
||||
An actor restart replaces only the actual actor object; the contents of the
|
||||
mailbox and the hotswap stack are unaffected by the restart, so processing of
|
||||
messages will resume after the :meth:`postRestart` hook returns. The message
|
||||
messages will resume after the :meth:`postRestart` hook returns. The message
|
||||
that triggered the exception will not be received again. Any message
|
||||
sent to an actor while it is being restarted will be queued to its mailbox as
|
||||
usual.
|
||||
|
|
@ -205,9 +205,9 @@ Stop Hook
|
|||
|
||||
After stopping an actor, its :meth:`postStop` hook is called, which may be used
|
||||
e.g. for deregistering this actor from other services. This hook is guaranteed
|
||||
to run after message queuing has been disabled for this actor, i.e. messages
|
||||
sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
|
||||
:obj:`ActorSystem`.
|
||||
to run after message queuing has been disabled for this actor, i.e. messages
|
||||
sent to a stopped actor will be redirected to the :obj:`deadLetters` of the
|
||||
:obj:`ActorSystem`.
|
||||
|
||||
|
||||
Identifying Actors
|
||||
|
|
@ -267,7 +267,7 @@ implicitly passed along with the message and available to the receiving Actor
|
|||
in its ``sender: ActorRef`` member field. The target actor can use this
|
||||
to reply to the original sender, by using ``sender ! replyMsg``.
|
||||
|
||||
If invoked from an instance that is **not** an Actor the sender will be
|
||||
If invoked from an instance that is **not** an Actor the sender will be
|
||||
:obj:`deadLetters` actor reference by default.
|
||||
|
||||
Ask: Send-And-Receive-Future
|
||||
|
|
@ -281,11 +281,11 @@ will immediately return a :class:`Future`:
|
|||
val future = actor ? "hello"
|
||||
|
||||
The receiving actor should reply to this message, which will complete the
|
||||
future with the reply message as value; ``sender ! result``.
|
||||
future with the reply message as value; ``sender ! result``.
|
||||
|
||||
To complete the future with an exception you need send a Failure message to the sender.
|
||||
This is not done automatically when an actor throws an exception while processing a
|
||||
message.
|
||||
To complete the future with an exception you need send a Failure message to the sender.
|
||||
This is not done automatically when an actor throws an exception while processing a
|
||||
message.
|
||||
|
||||
.. includecode:: code/ActorDocSpec.scala#reply-exception
|
||||
|
||||
|
|
@ -296,7 +296,7 @@ which is taken from one of the following locations in order of precedence:
|
|||
#. implicit argument of type :class:`akka.actor.Timeout`, e.g.
|
||||
|
||||
::
|
||||
|
||||
|
||||
import akka.actor.Timeout
|
||||
import akka.util.duration._
|
||||
|
||||
|
|
@ -306,16 +306,16 @@ which is taken from one of the following locations in order of precedence:
|
|||
See :ref:`futures-scala` for more information on how to await or query a
|
||||
future.
|
||||
|
||||
The ``onComplete``, ``onResult``, or ``onTimeout`` methods of the ``Future`` can be
|
||||
used to register a callback to get a notification when the Future completes.
|
||||
The ``onComplete``, ``onResult``, or ``onTimeout`` methods of the ``Future`` can be
|
||||
used to register a callback to get a notification when the Future completes.
|
||||
Gives you a way to avoid blocking.
|
||||
|
||||
.. warning::
|
||||
|
||||
When using future callbacks, inside actors you need to carefully avoid closing over
|
||||
the containing actor’s reference, i.e. do not call methods or access mutable state
|
||||
on the enclosing actor from within the callback. This would break the actor
|
||||
encapsulation and may introduce synchronization bugs and race conditions because
|
||||
the containing actor’s reference, i.e. do not call methods or access mutable state
|
||||
on the enclosing actor from within the callback. This would break the actor
|
||||
encapsulation and may introduce synchronization bugs and race conditions because
|
||||
the callback will be scheduled concurrently to the enclosing actor. Unfortunately
|
||||
there is not yet a way to detect these illegal accesses at compile time.
|
||||
See also: :ref:`jmm-shared-state`
|
||||
|
|
@ -403,17 +403,17 @@ object.
|
|||
Stopping actors
|
||||
===============
|
||||
|
||||
Actors are stopped by invoking the ``stop`` method of the ``ActorRef``.
|
||||
Actors are stopped by invoking the ``stop`` method of the ``ActorRef``.
|
||||
The actual termination of the actor is performed asynchronously, i.e.
|
||||
``stop`` may return before the actor is stopped.
|
||||
``stop`` may return before the actor is stopped.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
actor.stop()
|
||||
|
||||
Processing of the current message, if any, will continue before the actor is stopped,
|
||||
Processing of the current message, if any, will continue before the actor is stopped,
|
||||
but additional messages in the mailbox will not be processed. By default these
|
||||
messages are sent to the :obj:`deadLetters` of the :obj:`ActorSystem`, but that
|
||||
messages are sent to the :obj:`deadLetters` of the :obj:`ActorSystem`, but that
|
||||
depends on the mailbox implementation.
|
||||
|
||||
When stop is called then a call to the ``def postStop`` callback method will
|
||||
|
|
@ -540,11 +540,11 @@ messages on that mailbox, will be there as well.
|
|||
What happens to the actor
|
||||
-------------------------
|
||||
|
||||
If an exception is thrown, the actor instance is discarded and a new instance is
|
||||
If an exception is thrown, the actor instance is discarded and a new instance is
|
||||
created. This new instance will now be used in the actor references to this actor
|
||||
(so this is done invisible to the developer). Note that this means that current
|
||||
state of the failing actor instance is lost if you don't store and restore it in
|
||||
``preRestart`` and ``postRestart`` callbacks.
|
||||
(so this is done invisible to the developer). Note that this means that current
|
||||
state of the failing actor instance is lost if you don't store and restore it in
|
||||
``preRestart`` and ``postRestart`` callbacks.
|
||||
|
||||
|
||||
Extending Actors using PartialFunction chaining
|
||||
|
|
|
|||
|
|
@ -2,6 +2,7 @@ package akka.docs.actor
|
|||
|
||||
//#imports1
|
||||
import akka.actor.Actor
|
||||
import akka.actor.Props
|
||||
import akka.event.Logging
|
||||
//#imports1
|
||||
|
||||
|
|
@ -29,12 +30,12 @@ case class Message(s: String)
|
|||
|
||||
//#context-actorOf
|
||||
class FirstActor extends Actor {
|
||||
val myActor = context.actorOf[MyActor]
|
||||
val myActor = context.actorOf(Props[MyActor])
|
||||
//#context-actorOf
|
||||
//#anonymous-actor
|
||||
def receive = {
|
||||
case m: DoIt ⇒
|
||||
context.actorOf(new Actor {
|
||||
context.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case DoIt(msg) ⇒
|
||||
val replyMsg = doSomeDangerousWork(msg)
|
||||
|
|
@ -42,7 +43,7 @@ class FirstActor extends Actor {
|
|||
self.stop()
|
||||
}
|
||||
def doSomeDangerousWork(msg: ImmutableMessage): String = { "done" }
|
||||
}) ! m
|
||||
})) ! m
|
||||
|
||||
case replyMsg: String ⇒ sender ! replyMsg
|
||||
}
|
||||
|
|
@ -52,7 +53,7 @@ class FirstActor extends Actor {
|
|||
//#system-actorOf
|
||||
object Main extends App {
|
||||
val system = ActorSystem("MySystem")
|
||||
val myActor = system.actorOf[MyActor]
|
||||
val myActor = system.actorOf(Props[MyActor])
|
||||
//#system-actorOf
|
||||
}
|
||||
|
||||
|
|
@ -94,7 +95,7 @@ class Swapper extends Actor {
|
|||
|
||||
object SwapperApp extends App {
|
||||
val system = ActorSystem("SwapperSystem")
|
||||
val swap = system.actorOf[Swapper]
|
||||
val swap = system.actorOf(Props[Swapper])
|
||||
swap ! Swap // logs Hi
|
||||
swap ! Swap // logs Ho
|
||||
swap ! Swap // logs Hi
|
||||
|
|
@ -134,20 +135,20 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
//#import-context
|
||||
class FirstActor extends Actor {
|
||||
import context._
|
||||
val myActor = actorOf[MyActor]
|
||||
val myActor = actorOf(Props[MyActor])
|
||||
def receive = {
|
||||
case x ⇒ myActor ! x
|
||||
}
|
||||
}
|
||||
//#import-context
|
||||
|
||||
val first = system.actorOf(new FirstActor)
|
||||
val first = system.actorOf(Props(new FirstActor))
|
||||
first.stop()
|
||||
|
||||
}
|
||||
|
||||
"creating actor with AkkaSpec.actorOf" in {
|
||||
val myActor = system.actorOf[MyActor]
|
||||
val myActor = system.actorOf(Props[MyActor])
|
||||
|
||||
// testing the actor
|
||||
|
||||
|
|
@ -178,7 +179,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
|
||||
//#creating-constructor
|
||||
// allows passing in arguments to the MyActor constructor
|
||||
val myActor = system.actorOf(new MyActor("..."))
|
||||
val myActor = system.actorOf(Props(new MyActor("...")))
|
||||
//#creating-constructor
|
||||
|
||||
myActor.stop()
|
||||
|
|
@ -203,7 +204,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
}
|
||||
}
|
||||
|
||||
val myActor = system.actorOf(new MyActor)
|
||||
val myActor = system.actorOf(Props(new MyActor))
|
||||
implicit val timeout = system.settings.ActorTimeout
|
||||
val future = myActor ? "hello"
|
||||
future.as[String] match {
|
||||
|
|
@ -251,7 +252,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
}
|
||||
//#hot-swap-actor
|
||||
|
||||
val actor = system.actorOf(new HotSwapActor)
|
||||
val actor = system.actorOf(Props(new HotSwapActor))
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -26,8 +26,8 @@ To use it you can either create a Router through the ``routerActor()`` factory m
|
|||
|
||||
//Two actors, one named Pinger and one named Ponger
|
||||
//The actor(pf) method creates an anonymous actor and starts it
|
||||
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
val pinger = actorOf(Props(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(Props(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
|
||||
//A router that dispatches Ping messages to the pinger
|
||||
//and Pong messages to the ponger
|
||||
|
|
@ -53,8 +53,8 @@ Or by mixing in akka.routing.Router:
|
|||
|
||||
class MyRouter extends Actor with Router {
|
||||
//Our pinger and ponger actors
|
||||
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
val pinger = actorOf(Props(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(Props(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
//When we get a ping, we dispatch to the pinger
|
||||
//When we get a pong, we dispatch to the ponger
|
||||
def routes = {
|
||||
|
|
@ -64,7 +64,7 @@ Or by mixing in akka.routing.Router:
|
|||
}
|
||||
|
||||
//Create an instance of our router, and start it
|
||||
val d = actorOf[MyRouter]
|
||||
val d = actorOf(Props[MyRouter]
|
||||
|
||||
d ! Ping //Prints "Pinger: Ping"
|
||||
d ! Pong //Prints "Ponger: Pong"
|
||||
|
|
@ -90,8 +90,8 @@ Example using the ``loadBalancerActor()`` factory method:
|
|||
//Two actors, one named Pinger and one named Ponger
|
||||
//The actor(pf) method creates an anonymous actor and starts it
|
||||
|
||||
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
val pinger = actorOf(Props(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(Props(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
|
||||
//A load balancer that given a sequence of actors dispatches them accordingly
|
||||
//a CyclicIterator works in a round-robin-fashion
|
||||
|
|
@ -117,14 +117,14 @@ Or by mixing in akka.routing.LoadBalancer
|
|||
|
||||
//A load balancer that balances between a pinger and a ponger
|
||||
class MyLoadBalancer extends Actor with LoadBalancer {
|
||||
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
val pinger = actorOf(Props(new Actor { def receive = { case x => println("Pinger: " + x) } })
|
||||
val ponger = actorOf(Props(new Actor { def receive = { case x => println("Ponger: " + x) } })
|
||||
|
||||
val seq = new CyclicIterator[ActorRef](List(pinger,ponger))
|
||||
}
|
||||
|
||||
//Create an instance of our loadbalancer, and start it
|
||||
val d = actorOf[MyLoadBalancer]
|
||||
val d = actorOf(Props[MyLoadBalancer]
|
||||
|
||||
d ! Pong //Prints "Pinger: Pong"
|
||||
d ! Pong //Prints "Ponger: Pong"
|
||||
|
|
|
|||
|
|
@ -371,7 +371,7 @@ Here is an example of using ``retry`` to block until an account has enough money
|
|||
val account1 = Ref(100.0)
|
||||
val account2 = Ref(100.0)
|
||||
|
||||
val transferer = Actor.actorOf(new Transferer)
|
||||
val transferer = Actor.actorOf(Props(new Transferer)
|
||||
|
||||
transferer ! Transfer(account1, account2, 500.0)
|
||||
// INFO Transferer: not enough money - retrying
|
||||
|
|
@ -427,7 +427,7 @@ You can also have two alternative blocking transactions, one of which can succee
|
|||
val ref1 = Ref(0)
|
||||
val ref2 = Ref(0)
|
||||
|
||||
val brancher = Actor.actorOf(new Brancher)
|
||||
val brancher = Actor.actorOf(Props(new Brancher)
|
||||
|
||||
brancher ! Branch(ref1, ref2, 1)
|
||||
// INFO Brancher: not enough on left - retrying
|
||||
|
|
|
|||
|
|
@ -106,13 +106,13 @@ and in addition allows access to the internal state::
|
|||
case Ev("back") => goto(1) using "back"
|
||||
}
|
||||
})
|
||||
|
||||
|
||||
assert (fsm.stateName == 1)
|
||||
assert (fsm.stateData == "")
|
||||
fsm ! "go" // being a TestActorRef, this runs also on the CallingThreadDispatcher
|
||||
assert (fsm.stateName == 2)
|
||||
assert (fsm.stateData == "go")
|
||||
|
||||
|
||||
fsm.setState(stateName = 1)
|
||||
assert (fsm.stateName == 1)
|
||||
|
||||
|
|
@ -235,8 +235,8 @@ common task easy:
|
|||
"An Echo actor" must {
|
||||
|
||||
"send back messages unchanged" in {
|
||||
|
||||
val echo = Actor.actorOf[EchoActor]
|
||||
|
||||
val echo = Actor.actorOf(Props[EchoActor]
|
||||
echo ! "hello world"
|
||||
expectMsg("hello world")
|
||||
|
||||
|
|
@ -352,11 +352,11 @@ with message flows:
|
|||
* :meth:`receiveWhile[T](max: Duration, idle: Duration)(pf: PartialFunction[Any, T]): Seq[T]`
|
||||
|
||||
Collect messages as long as
|
||||
|
||||
|
||||
* they are matching the given partial function
|
||||
* the given time interval is not used up
|
||||
* the next message is received within the idle timeout
|
||||
|
||||
|
||||
All collected messages are returned. The maximum duration defaults to the
|
||||
time remaining in the innermost enclosing :ref:`within <TestKit.within>`
|
||||
block and the idle duration defaults to infinity (thereby disabling the
|
||||
|
|
@ -370,7 +370,7 @@ with message flows:
|
|||
:ref:`within <TestKit.within>` block.
|
||||
|
||||
* :meth:`ignoreMsg(pf: PartialFunction[AnyRef, Boolean])`
|
||||
|
||||
|
||||
:meth:`ignoreNoMsg`
|
||||
|
||||
The internal :obj:`testActor` contains a partial function for ignoring
|
||||
|
|
@ -506,7 +506,7 @@ using a small example::
|
|||
|
||||
val probe1 = TestProbe()
|
||||
val probe2 = TestProbe()
|
||||
val actor = Actor.actorOf[MyDoubleEcho]
|
||||
val actor = Actor.actorOf(Props[MyDoubleEcho]
|
||||
actor ! (probe1.ref, probe2.ref)
|
||||
actor ! "hello"
|
||||
probe1.expectMsg(50 millis, "hello")
|
||||
|
|
@ -553,8 +553,8 @@ concerning volume and timing of the message flow while still keeping the
|
|||
network functioning::
|
||||
|
||||
val probe = TestProbe()
|
||||
val source = Actor.actorOf(new Source(probe))
|
||||
val dest = Actor.actorOf[Destination]
|
||||
val source = Actor.actorOf(Props(new Source(probe))
|
||||
val dest = Actor.actorOf(Props[Destination]
|
||||
source ! "start"
|
||||
probe.expectMsg("work")
|
||||
probe.forward(dest)
|
||||
|
|
@ -710,7 +710,7 @@ by debuggers as well as logging, where the Akka toolkit offers the following
|
|||
options:
|
||||
|
||||
* *Logging of exceptions thrown within Actor instances*
|
||||
|
||||
|
||||
This is always on; in contrast to the other logging mechanisms, this logs at
|
||||
``ERROR`` level.
|
||||
|
||||
|
|
@ -723,7 +723,7 @@ options:
|
|||
import akka.event.LoggingReceive
|
||||
def receive = LoggingReceive(this) {
|
||||
case msg => ...
|
||||
}
|
||||
}
|
||||
|
||||
The first argument to :meth:`LoggingReceive` defines the source to be used in the
|
||||
logging events, which should be the current actor.
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
.. code-block:: scala
|
||||
|
||||
package unit.akka
|
||||
|
||||
|
||||
import org.scalatest.matchers.ShouldMatchers
|
||||
import org.scalatest.{WordSpec, BeforeAndAfterAll}
|
||||
import akka.actor.Actor._
|
||||
|
|
@ -18,21 +18,21 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
import java.util.concurrent.TimeUnit
|
||||
import akka.actor.{ActorRef, Actor}
|
||||
import util.Random
|
||||
|
||||
|
||||
/**
|
||||
* a Test to show some TestKit examples
|
||||
*/
|
||||
|
||||
|
||||
class TestKitUsageSpec extends WordSpec with BeforeAndAfterAll with ShouldMatchers with TestKit {
|
||||
val echoRef = actorOf(new EchoActor)
|
||||
val forwardRef = actorOf(new ForwardingActor(testActor))
|
||||
val filterRef = actorOf(new FilteringActor(testActor))
|
||||
val echoRef = actorOf(Props(new EchoActor)
|
||||
val forwardRef = actorOf(Props(new ForwardingActor(testActor))
|
||||
val filterRef = actorOf(Props(new FilteringActor(testActor))
|
||||
val randomHead = Random.nextInt(6)
|
||||
val randomTail = Random.nextInt(10)
|
||||
val headList = List().padTo(randomHead, "0")
|
||||
val tailList = List().padTo(randomTail, "1")
|
||||
val seqRef = actorOf(new SequencingActor(testActor, headList, tailList))
|
||||
|
||||
val seqRef = actorOf(Props(new SequencingActor(testActor, headList, tailList))
|
||||
|
||||
override protected def afterAll(): scala.Unit = {
|
||||
stopTestActor
|
||||
echoRef.stop()
|
||||
|
|
@ -40,7 +40,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
filterRef.stop()
|
||||
seqRef.stop()
|
||||
}
|
||||
|
||||
|
||||
"An EchoActor" should {
|
||||
"Respond with the same message it receives" in {
|
||||
within(100 millis) {
|
||||
|
|
@ -70,7 +70,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
filterRef ! 1
|
||||
filterRef ! "text"
|
||||
filterRef ! 1
|
||||
|
||||
|
||||
receiveWhile(500 millis) {
|
||||
case msg: String => messages = msg :: messages
|
||||
}
|
||||
|
|
@ -95,7 +95,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* An Actor that echoes everything you send to it
|
||||
*/
|
||||
|
|
@ -106,7 +106,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* An Actor that forwards every message to a next Actor
|
||||
*/
|
||||
|
|
@ -117,7 +117,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* An Actor that only forwards certain messages to a next Actor
|
||||
*/
|
||||
|
|
@ -129,7 +129,7 @@ Ray Roestenburg's example code from `his blog <http://roestenburg.agilesquad.com
|
|||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* An actor that sends a sequence of messages with a random head list, an interesting value and a random tail list
|
||||
* The idea is that you would like to test that the interesting value is received and that you cant be bothered with the rest
|
||||
|
|
|
|||
|
|
@ -64,8 +64,8 @@ Here is an example of coordinating two simple counter Actors so that they both i
|
|||
}
|
||||
}
|
||||
|
||||
val counter1 = Actor.actorOf[Counter]
|
||||
val counter2 = Actor.actorOf[Counter]
|
||||
val counter1 = Actor.actorOf(Props[Counter]
|
||||
val counter2 = Actor.actorOf(Props[Counter]
|
||||
|
||||
counter1 ! Coordinated(Increment(Some(counter2)))
|
||||
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import org.bson.DefaultBSONSerializer
|
|||
import akka.actor.SerializedActorRef
|
||||
import akka.remote.RemoteProtocol.MessageProtocol
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl, Props }
|
||||
|
||||
class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
|
||||
|
|
@ -71,7 +71,7 @@ class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObjec
|
|||
val msg = MessageSerializer.deserialize(system, msgData)
|
||||
val ownerPath = doc.as[String]("ownerPath")
|
||||
val senderPath = doc.as[String]("senderPath")
|
||||
val sender = systemImpl.actorOf(senderPath)
|
||||
val sender = systemImpl.actorFor(senderPath)
|
||||
|
||||
MongoDurableMessage(ownerPath, msg, sender)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb", MongoDurableMa
|
|||
}
|
||||
|
||||
def createMongoMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef = {
|
||||
val queueActor = actorOf[MongoMailboxTestActor]
|
||||
val queueActor = actorOf(Props[MongoMailboxTestActor]
|
||||
queueActor.dispatcher = dispatcher
|
||||
queueActor
|
||||
}
|
||||
|
|
@ -47,7 +47,7 @@ class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb", MongoDurableMa
|
|||
"should handle reply to ! for 1 message" in {
|
||||
val latch = new CountDownLatch(1)
|
||||
val queueActor = createMongoMailboxTestActor("mongoDB Backend should handle Reply to !")
|
||||
val sender = actorOf(new Actor { def receive = { case "sum" => latch.countDown } })
|
||||
val sender = actorOf(Props(new Actor { def receive = { case "sum" => latch.countDown } })
|
||||
|
||||
queueActor.!("sum")(Some(sender))
|
||||
latch.await(10, TimeUnit.SECONDS) must be (true)
|
||||
|
|
|
|||
|
|
@ -59,7 +59,7 @@
|
|||
// remote.start()
|
||||
|
||||
// barrier("start")
|
||||
// val actor = system.actorOf[SomeActor]("service-hello")
|
||||
// val actor = system.actorOf(Props[SomeActor]("service-hello")
|
||||
// actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
// val connectionCount = NrOfNodes - 1
|
||||
|
|
|
|||
|
|
@ -47,7 +47,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec with DefaultTi
|
|||
|
||||
barrier("start")
|
||||
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf(Props[SomeActor]("service-hello")
|
||||
//actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val result = (actor ? "identify").get
|
||||
|
|
|
|||
|
|
@ -47,7 +47,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec with DefaultTimeout {
|
|||
|
||||
barrier("start")
|
||||
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf(Props[SomeActor]("service-hello")
|
||||
val result = (actor ? "identify").get
|
||||
result must equal("node1")
|
||||
|
||||
|
|
|
|||
|
|
@ -71,7 +71,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec with DefaultTi
|
|||
remote.start()
|
||||
|
||||
barrier("start")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf(Props[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = NrOfNodes - 1
|
||||
|
|
|
|||
|
|
@ -71,7 +71,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec with Defau
|
|||
remote.start()
|
||||
|
||||
barrier("start")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf(Props[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = NrOfNodes - 1
|
||||
|
|
|
|||
|
|
@ -71,7 +71,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec with De
|
|||
remote.start()
|
||||
|
||||
barrier("start")
|
||||
val actor = system.actorOf[SomeActor]("service-hello")
|
||||
val actor = system.actorOf(Props[SomeActor]("service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true)
|
||||
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package sample.fsm.dining.become
|
|||
//Akka adaptation of
|
||||
//http://www.dalnefre.com/wp/2010/08/dining-philosophers-in-humus/
|
||||
|
||||
import akka.actor.{ ActorRef, Actor, ActorSystem }
|
||||
import akka.actor._
|
||||
import akka.util.duration._
|
||||
|
||||
/*
|
||||
|
|
@ -137,12 +137,12 @@ object DiningHakkers {
|
|||
|
||||
def run {
|
||||
//Create 5 chopsticks
|
||||
val chopsticks = for (i ← 1 to 5) yield system.actorOf[Chopstick]("Chopstick " + i)
|
||||
val chopsticks = for (i ← 1 to 5) yield system.actorOf(Props[Chopstick], "Chopstick " + i)
|
||||
|
||||
//Create 5 awesome hakkers and assign them their left and right chopstick
|
||||
val hakkers = for {
|
||||
(name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex
|
||||
} yield system.actorOf(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5)))
|
||||
} yield system.actorOf(Props(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5))))
|
||||
|
||||
//Signal all hakkers that they should start thinking, and watch the show
|
||||
hakkers.foreach(_ ! Think)
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package sample.fsm.dining.fsm
|
||||
|
||||
import akka.actor.{ ActorRef, Actor, FSM, ActorSystem }
|
||||
import akka.actor._
|
||||
import akka.actor.FSM._
|
||||
import akka.util.Duration
|
||||
import akka.util.duration._
|
||||
|
|
@ -175,11 +175,11 @@ object DiningHakkersOnFsm {
|
|||
|
||||
def run = {
|
||||
// Create 5 chopsticks
|
||||
val chopsticks = for (i ← 1 to 5) yield system.actorOf[Chopstick]("Chopstick " + i)
|
||||
val chopsticks = for (i ← 1 to 5) yield system.actorOf(Props[Chopstick], "Chopstick " + i)
|
||||
// Create 5 awesome fsm hakkers and assign them their left and right chopstick
|
||||
val hakkers = for {
|
||||
(name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex
|
||||
} yield system.actorOf(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5)))
|
||||
} yield system.actorOf(Props(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5))))
|
||||
|
||||
hakkers.foreach(_ ! Think)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,19 +3,19 @@
|
|||
*/
|
||||
package sample.hello
|
||||
|
||||
import akka.actor.{ ActorSystem, Actor }
|
||||
import akka.actor.{ ActorSystem, Actor, Props }
|
||||
|
||||
case object Start
|
||||
|
||||
object Main {
|
||||
def main(args: Array[String]): Unit = {
|
||||
val system = ActorSystem()
|
||||
system.actorOf[HelloActor] ! Start
|
||||
system.actorOf(Props[HelloActor]) ! Start
|
||||
}
|
||||
}
|
||||
|
||||
class HelloActor extends Actor {
|
||||
val worldActor = context.actorOf[WorldActor]
|
||||
val worldActor = context.actorOf(Props[WorldActor])
|
||||
def receive = {
|
||||
case Start ⇒ worldActor ! "Hello"
|
||||
case s: String ⇒
|
||||
|
|
|
|||
|
|
@ -94,7 +94,7 @@ object TestActorRef {
|
|||
"Could not instantiate Actor" +
|
||||
"\nMake sure Actor is NOT defined inside a class/trait," +
|
||||
"\nif so put it outside the class/trait, f.e. in a companion object," +
|
||||
"\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", exception)
|
||||
"\nOR try to change: 'actorOf(Props[MyActor]' to 'actorOf(Props(new MyActor)'.", exception)
|
||||
}
|
||||
}), name)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -54,7 +54,7 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
|
|||
*
|
||||
* <pre>
|
||||
* class Test extends TestKit {
|
||||
* val test = actorOf[SomeActor]
|
||||
* val test = actorOf(Props[SomeActor]
|
||||
*
|
||||
* within (1 second) {
|
||||
* test ! SomeWork
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
|
||||
package akka.tutorial.first.java;
|
||||
|
||||
import akka.actor.Props;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.InternalActorRef;
|
||||
|
|
@ -112,7 +113,7 @@ public class Pi {
|
|||
};
|
||||
LinkedList<ActorRef> actors = new LinkedList<ActorRef>() {
|
||||
{
|
||||
for (int i = 0; i < nrOfWorkers; i++) add(getContext().actorOf(Worker.class));
|
||||
for (int i = 0; i < nrOfWorkers; i++) add(getContext().actorOf(new Props(Worker.class)));
|
||||
}
|
||||
};
|
||||
// FIXME routers are intended to be used like this
|
||||
|
|
@ -166,11 +167,11 @@ public class Pi {
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
// create the master
|
||||
ActorRef master = system.actorOf(new UntypedActorFactory() {
|
||||
ActorRef master = system.actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
// start the calculation
|
||||
master.tell(new Calculate());
|
||||
|
|
|
|||
|
|
@ -5,8 +5,7 @@ package akka.tutorial.first.scala
|
|||
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import akka.routing.{ RoutedActorRef, LocalConnectionManager, RoundRobinRouter, RoutedProps }
|
||||
import akka.actor.{ ActorSystemImpl, Actor, ActorSystem }
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.actor._
|
||||
|
||||
object Pi extends App {
|
||||
|
||||
|
|
@ -53,7 +52,7 @@ object Pi extends App {
|
|||
var start: Long = _
|
||||
|
||||
// create the workers
|
||||
val workers = Vector.fill(nrOfWorkers)(context.actorOf[Worker])
|
||||
val workers = Vector.fill(nrOfWorkers)(context.actorOf(Props[Worker]))
|
||||
|
||||
// wrap them with a load-balancing router
|
||||
// FIXME routers are intended to be used like this
|
||||
|
|
@ -99,7 +98,7 @@ object Pi extends App {
|
|||
val latch = new CountDownLatch(1)
|
||||
|
||||
// create the master
|
||||
val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
|
||||
val master = system.actorOf(Props(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)))
|
||||
|
||||
// start the calculation
|
||||
master ! Calculate
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue