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:
Jonas Bonér 2011-12-13 14:09:40 +01:00
parent 86a5114d79
commit c9b787f029
85 changed files with 464 additions and 518 deletions

View file

@ -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);
}

View file

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

View file

@ -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

View file

@ -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()

View file

@ -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))

View file

@ -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))

View file

@ -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))

View file

@ -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)

View file

@ -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"

View file

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

View file

@ -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 ! ""
}

View file

@ -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()

View file

@ -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)

View file

@ -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) {

View file

@ -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

View file

@ -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]

View file

@ -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 {
}
}
}

View file

@ -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"

View file

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

View file

@ -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");
* }}}

View file

@ -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>

View file

@ -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

View file

@ -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");
* }}}

View file

@ -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

View file

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

View file

@ -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]

View file

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

View file

@ -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)

View file

@ -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

View file

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

View file

@ -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] {

View file

@ -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)

View file

@ -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) {

View file

@ -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

View file

@ -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]"))

View file

@ -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

View file

@ -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) {

View file

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

View file

@ -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

View file

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

View file

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

View file

@ -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)

View file

@ -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)

View file

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

View file

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

View file

@ -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)

View file

@ -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)

View file

@ -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)

View file

@ -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)

View file

@ -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

View file

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

View file

@ -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 = {

View file

@ -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

View file

@ -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);
}

View file

@ -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))

View file

@ -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.

View file

@ -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

View file

@ -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

View file

@ -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");

View file

@ -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

View file

@ -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) {

View file

@ -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]

View file

@ -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)

View file

@ -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 actors 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 actors 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

View file

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

View file

@ -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"

View file

@ -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

View file

@ -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.

View file

@ -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

View file

@ -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)))

View file

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

View file

@ -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)

View file

@ -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

View file

@ -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

View file

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

View file

@ -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

View file

@ -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

View file

@ -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)

View file

@ -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)

View file

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

View file

@ -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

View file

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

View file

@ -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

View file

@ -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());

View file

@ -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