Start removed but cluster is broken

This commit is contained in:
Viktor Klang 2011-09-08 11:02:17 +02:00
parent 24fb967988
commit bbb79d8e0d
144 changed files with 684 additions and 930 deletions

View file

@ -67,16 +67,16 @@ class ActorFireForgetRequestReplySpec extends WordSpec with MustMatchers with Be
"An Actor" must {
"reply to bang message using reply" in {
val replyActor = actorOf[ReplyActor].start()
val senderActor = actorOf(new SenderActor(replyActor)).start()
val replyActor = actorOf[ReplyActor]
val senderActor = actorOf(new SenderActor(replyActor))
senderActor ! "Init"
state.finished.await
state.s must be("Reply")
}
"reply to bang message using implicit sender" in {
val replyActor = actorOf[ReplyActor].start()
val senderActor = actorOf(new SenderActor(replyActor)).start()
val replyActor = actorOf[ReplyActor]
val senderActor = actorOf(new SenderActor(replyActor))
senderActor ! "InitImplicit"
state.finished.await
state.s must be("ReplyImplicit")
@ -84,7 +84,7 @@ class ActorFireForgetRequestReplySpec extends WordSpec with MustMatchers with Be
"should shutdown crashed temporary actor" in {
filterEvents(EventFilter[Exception]("Expected")) {
val actor = actorOf(Props[CrashingActor].withLifeCycle(Temporary)).start()
val actor = actorOf(Props[CrashingActor].withLifeCycle(Temporary))
actor.isRunning must be(true)
actor ! "Die"
state.finished.await

View file

@ -126,7 +126,7 @@ class ActorRefSpec extends WordSpec with MustMatchers {
actorOf(new Actor {
val nested = new Actor { def receive = { case _ } }
def receive = { case _ }
}).start()
})
}
def refStackMustBeEmpty = Actor.actorRefInCreation.get.headOption must be === None
@ -134,37 +134,37 @@ class ActorRefSpec extends WordSpec with MustMatchers {
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new FailingOuterActor(actorOf(new InnerActor).start)).start()
actorOf(new FailingOuterActor(actorOf(new InnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new OuterActor(actorOf(new FailingInnerActor).start)).start()
actorOf(new OuterActor(actorOf(new FailingInnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new FailingInheritingOuterActor(actorOf(new InnerActor).start)).start()
actorOf(new FailingInheritingOuterActor(actorOf(new InnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new FailingInheritingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new FailingInheritingOuterActor(actorOf(new FailingInheritingInnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new FailingInheritingOuterActor(actorOf(new FailingInnerActor).start)).start()
actorOf(new FailingInheritingOuterActor(actorOf(new FailingInnerActor)))
}
refStackMustBeEmpty
@ -172,38 +172,38 @@ class ActorRefSpec extends WordSpec with MustMatchers {
intercept[akka.actor.ActorInitializationException] {
actorOf(new OuterActor(actorOf(new InnerActor {
val a = new InnerActor
}).start)).start()
})))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new OuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new OuterActor(actorOf(new FailingInheritingInnerActor)))
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
actorOf(new OuterActor(actorOf({ new InnerActor; new InnerActor }).start)).start()
actorOf(new OuterActor(actorOf({ new InnerActor; new InnerActor })))
}
refStackMustBeEmpty
(intercept[java.lang.IllegalStateException] {
actorOf(new OuterActor(actorOf({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor }).start)).start()
actorOf(new OuterActor(actorOf({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })))
}).getMessage must be === "Ur state be b0rked"
refStackMustBeEmpty
}
"be serializable using Java Serialization on local node" in {
val a = actorOf[InnerActor].start
val a = actorOf[InnerActor]
import java.io._
@ -226,7 +226,7 @@ class ActorRefSpec extends WordSpec with MustMatchers {
"must throw exception on deserialize if not present in local registry and remoting is not enabled" in {
ReflectiveAccess.RemoteModule.isEnabled must be === false
val a = actorOf[InnerActor].start
val a = actorOf[InnerActor]
val inetAddress = ReflectiveAccess.RemoteModule.configDefaultAddress
@ -257,9 +257,9 @@ class ActorRefSpec extends WordSpec with MustMatchers {
"support nested actorOfs" in {
val a = actorOf(new Actor {
val nested = actorOf(new Actor { def receive = { case _ } }).start()
val nested = actorOf(new Actor { def receive = { case _ } })
def receive = { case _ self reply nested }
}).start()
})
val nested = (a ? "any").as[ActorRef].get
a must not be null

View file

@ -1,7 +1,7 @@
package akka.actor
import org.scalatest.junit.JUnitSuite
import org.scalatest.BeforeAndAfterAll
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import org.junit.Test
import Actor._
import org.scalatest.Assertions._
@ -30,14 +30,13 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
import ActorRegistrySpec._
override def afterAll = {
akka.event.EventHandler.start
akka.event.EventHandler.start()
}
@Test
def shouldGetActorByAddressFromActorRegistry {
Actor.registry.local.shutdownAll
val actor1 = actorOf[TestActor]("test-actor-1")
actor1.start
val actor2 = Actor.registry.actorFor(actor1.address)
assert(actor2.isDefined)
assert(actor2.get.address === actor1.address)
@ -51,7 +50,6 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
Actor.registry.local.shutdownAll
val actor = actorOf[TestActor]("test-actor-1")
val uuid = actor.uuid
actor.start
val actorOrNone = Actor.registry.local.actorFor(uuid)
assert(actorOrNone.isDefined)
assert(actorOrNone.get.uuid === uuid)
@ -63,7 +61,7 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldFindThingsFromLocalActorRegistry {
Actor.registry.local.shutdownAll
val actor = actorOf[TestActor]("test-actor-1").start()
val actor = actorOf[TestActor]("test-actor-1")
val found: Option[LocalActorRef] = Actor.registry.local.find({ case a: LocalActorRef if a.actorInstance.get().isInstanceOf[TestActor] a })
assert(found.isDefined)
assert(found.get.actorInstance.get().isInstanceOf[TestActor])
@ -74,14 +72,12 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldGetAllActorsFromLocalActorRegistry {
Actor.registry.local.shutdownAll
val actor1 = actorOf[TestActor]("test-actor-1").start()
val actor2 = actorOf[TestActor]("test-actor-2").start()
val actor1 = actorOf[TestActor]("test-actor-1")
val actor2 = actorOf[TestActor]("test-actor-2")
val actors = Actor.registry.local.actors
assert(actors.size === 2)
assert(actors.head.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor])
assert(actors.head.address === "test-actor-2")
assert(actors.last.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor])
assert(actors.last.address === "test-actor-1")
assert(actors.find(_.address == "test-actor-2").get.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor])
assert(actors.find(_.address == "test-actor-1").get.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor])
actor1.stop
actor2.stop
}
@ -89,8 +85,8 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldGetResponseByAllActorsInLocalActorRegistryWhenInvokingForeach {
Actor.registry.local.shutdownAll
val actor1 = actorOf[TestActor]("test-actor-1").start
val actor2 = actorOf[TestActor]("test-actor-2").start
val actor1 = actorOf[TestActor]("test-actor-1")
val actor2 = actorOf[TestActor]("test-actor-2")
val results = new ConcurrentLinkedQueue[Future[String]]
Actor.registry.local.foreach(actor results.add(actor.?("ping").mapTo[String]))
@ -106,9 +102,7 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
def shouldShutdownAllActorsInLocalActorRegistry {
Actor.registry.local.shutdownAll
val actor1 = actorOf[TestActor]("test-actor-1")
actor1.start
val actor2 = actorOf[TestActor]("test-actor-2")
actor2.start
Actor.registry.local.shutdownAll
assert(Actor.registry.local.actors.size === 0)
}
@ -117,50 +111,11 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
def shouldRemoveUnregisterActorInLocalActorRegistry {
Actor.registry.local.shutdownAll
val actor1 = actorOf[TestActor]("test-actor-1")
actor1.start
val actor2 = actorOf[TestActor]("test-actor-2")
actor2.start
assert(Actor.registry.local.actors.size === 2)
Actor.registry.unregister(actor1)
assert(Actor.registry.local.actors.size === 1)
Actor.registry.unregister(actor2)
assert(Actor.registry.local.actors.size === 0)
}
/*
@Test def shouldBeAbleToRegisterActorsConcurrently {
Actor.registry.local.shutdownAll
def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor {
self.address = i.toString
def receive = { case _ => }
})
val latch = new CountDownLatch(3)
val barrier = new CyclicBarrier(3)
def mkThread(actors: Iterable[ActorRef]) = new Thread {
this.start()
override def run {
barrier.await
actors foreach { _.start() }
latch.countDown()
}
}
val a1,a2,a3 = mkTestActors
val t1 = mkThread(a1)
val t2 = mkThread(a2)
val t3 = mkThread(a3)
assert(latch.await(30,TimeUnit.SECONDS) === true)
for(i <- 1 to 10) {
val theId = i.toString
val actor = Actor.registry.local.actorFor(theId)
assert(actor eq a)
assert(actors.size === 9000)
}
}
*/
}

View file

@ -45,7 +45,7 @@ object ActorRestartSpec {
val ref = TestActorRef(new Actor {
def receive = { case _ }
override def preStart { testActor ! ((this, self)) }
}).start()
})
testActor ! ((ref.underlyingActor, ref))
None
case Handover
@ -81,7 +81,7 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo
private def collect(f: ActorRef): ActorRef = {
val ref = f
toStop add ref
ref.start()
ref
}
private def createSupervisor =

View file

@ -19,7 +19,7 @@ class ActorTimeoutSpec
def receive = {
case x
}
}, timeout = t)).start()
}, timeout = t))
val testTimeout = if (Timeout.default.duration < 400.millis) 500 millis else 100 millis

View file

@ -28,7 +28,6 @@ object Chameneos {
class Chameneo(var mall: ActorRef, var colour: Colour, cid: Int) extends Actor {
var meetings = 0
self.start()
mall ! Meet(self, colour)
def receive = {
@ -110,7 +109,7 @@ object Chameneos {
def run {
// System.setProperty("akka.config", "akka.conf")
Chameneos.start = System.currentTimeMillis
actorOf(new Mall(1000000, 4)).start()
actorOf(new Mall(1000000, 4))
Thread.sleep(10000)
println("Elapsed: " + (end - start))
}

View file

@ -29,7 +29,7 @@ class ChannelSpec extends WordSpec with MustMatchers {
def receive = {
case str: String ch ! str
}
}).start()
})
a ! "hallo"
s must be(("hallo", a))

View file

@ -134,14 +134,14 @@ class FSMActorSpec extends WordSpec with MustMatchers with TestKit with BeforeAn
"unlock the lock" in {
// lock that locked after being open for 1 sec
val lock = Actor.actorOf(new Lock("33221", 1 second)).start()
val lock = Actor.actorOf(new Lock("33221", 1 second))
val transitionTester = Actor.actorOf(new Actor {
def receive = {
case Transition(_, _, _) transitionCallBackLatch.open
case CurrentState(_, Locked) initialStateLatch.open
}
}).start()
})
lock ! SubscribeTransitionCallBack(transitionTester)
initialStateLatch.await
@ -174,7 +174,7 @@ class FSMActorSpec extends WordSpec with MustMatchers with TestKit with BeforeAn
case "world" answerLatch.open
case Bye lock ! "bye"
}
}).start()
})
tester ! Hello
answerLatch.await
@ -188,7 +188,7 @@ class FSMActorSpec extends WordSpec with MustMatchers with TestKit with BeforeAn
when(1) {
case Ev("go") goto(2)
}
}).start()
})
logger = Actor.actorOf(new Actor {
def receive = {
case x testActor forward x
@ -209,7 +209,7 @@ class FSMActorSpec extends WordSpec with MustMatchers with TestKit with BeforeAn
case x testActor ! x
}
}
val ref = Actor.actorOf(fsm).start()
val ref = Actor.actorOf(fsm)
ref.stop()
expectMsg(1 second, fsm.StopEvent(Shutdown, 1, null))
}
@ -230,7 +230,7 @@ class FSMActorSpec extends WordSpec with MustMatchers with TestKit with BeforeAn
onTermination {
case StopEvent(r, _, _) testActor ! r
}
}).start()
})
val fsm = fsmref.underlyingActor
logger = Actor.actorOf(new Actor {
def receive = {
@ -261,7 +261,7 @@ class FSMActorSpec extends WordSpec with MustMatchers with TestKit with BeforeAn
case Event("count", c) stay using (c + 1)
case Event("log", _) stay replying getLog
}
}).start()
})
fsmref ! "log"
val fsm = fsmref.underlyingActor
expectMsg(1 second, IndexedSeq(LogEntry(1, 0, "log")))

View file

@ -15,7 +15,7 @@ class FSMTimingSpec extends WordSpec with MustMatchers with TestKit {
import FSMTimingSpec._
import FSM._
val fsm = Actor.actorOf(new StateMachine(testActor)).start()
val fsm = Actor.actorOf(new StateMachine(testActor))
fsm ! SubscribeTransitionCallBack(testActor)
expectMsg(200 millis, CurrentState(fsm, Initial))

View file

@ -48,7 +48,7 @@ class FSMTransitionSpec extends WordSpec with MustMatchers with TestKit {
"A FSM transition notifier" must {
"notify listeners" in {
val fsm = Actor.actorOf(new MyFSM(testActor)).start()
val fsm = Actor.actorOf(new MyFSM(testActor))
within(1 second) {
fsm ! SubscribeTransitionCallBack(testActor)
expectMsg(CurrentState(fsm, 0))
@ -60,8 +60,8 @@ class FSMTransitionSpec extends WordSpec with MustMatchers with TestKit {
}
"not fail when listener goes away" in {
val forward = Actor.actorOf(new Forwarder(testActor)).start()
val fsm = Actor.actorOf(new MyFSM(testActor)).start()
val forward = Actor.actorOf(new Forwarder(testActor))
val fsm = Actor.actorOf(new MyFSM(testActor))
val sup = Actor.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
sup link fsm
within(300 millis) {
@ -72,27 +72,6 @@ class FSMTransitionSpec extends WordSpec with MustMatchers with TestKit {
expectNoMsg
}
}
"not fail when listener is invalid" in {
val forward = Actor.actorOf(new Forwarder(testActor))
val fsm = Actor.actorOf(new MyFSM(testActor)).start()
val sup = Actor.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
sup link fsm
within(300 millis) {
filterEvents(EventFilter.custom {
case EventHandler.Warning(_: MyFSM, _) true
case _ false
}) {
fsm ! SubscribeTransitionCallBack(forward)
fsm ! "reply"
expectMsg("reply")
}
forward.start()
fsm ! SubscribeTransitionCallBack(forward)
expectMsg(CurrentState(fsm, 0))
}
}
}
}

View file

@ -19,12 +19,12 @@ object ForwardActorSpec {
def createForwardingChain(): ActorRef = {
val replier = actorOf(new Actor {
def receive = { case x self reply x }
}).start()
})
def mkforwarder(forwardTo: ActorRef) = actorOf(
new Actor {
def receive = { case x forwardTo forward x }
}).start()
})
mkforwarder(mkforwarder(mkforwarder(replier)))
}
@ -38,7 +38,7 @@ class ForwardActorSpec extends WordSpec with MustMatchers {
"forward actor reference when invoking forward on bang" in {
val latch = new TestLatch(1)
val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } }).start()
val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } })
val chain = createForwardingChain()

View file

@ -21,7 +21,7 @@ class HotSwapSpec extends WordSpec with MustMatchers {
var _log = ""
val a = actorOf(new Actor {
def receive = { case _ _log += "default" }
}).start()
})
a ! HotSwap(self {
case _
_log += "swapped"
@ -47,7 +47,7 @@ class HotSwapSpec extends WordSpec with MustMatchers {
barrier.await
})
}
}).start()
})
a ! "init"
barrier.await
@ -71,7 +71,7 @@ class HotSwapSpec extends WordSpec with MustMatchers {
_log += "init"
barrier.await
}
}).start()
})
a ! "init"
barrier.await
@ -126,7 +126,7 @@ class HotSwapSpec extends WordSpec with MustMatchers {
})
barrier.await
}
}).start()
})
a ! "init"
barrier.await

View file

@ -22,7 +22,7 @@ object IOActorSpec {
listen(ioManager, host, port)
}
def createWorker = Actor.actorOf(new Actor with IO {
def createWorker = Actor.actorOf(Props(new Actor with IO {
def receiveIO = {
case NewClient(server)
val socket = server.accept()
@ -31,11 +31,11 @@ object IOActorSpec {
socket write bytes
}
}
})
}).withSupervisor(optionSelf))
def receive = {
case msg: NewClient
self.link(createWorker).start() forward msg
createWorker forward msg
}
}
@ -51,7 +51,7 @@ object IOActorSpec {
self reply bytes
}
}
}.start
}
def receiveIO = {
case bytes: ByteString
@ -69,7 +69,7 @@ object IOActorSpec {
listen(ioManager, host, port)
}
def createWorker = Actor.actorOf(new Actor with IO {
def createWorker = Actor.actorOf(Props(new Actor with IO {
def receiveIO = {
case NewClient(server)
val socket = server.accept()
@ -101,10 +101,10 @@ object IOActorSpec {
}
}
}
})
}).withSupervisor(self))
def receive = {
case msg: NewClient self.link(createWorker).start() forward msg
case msg: NewClient createWorker forward msg
case ('set, key: String, value: ByteString)
kvs += (key -> value)
self tryReply (())
@ -172,9 +172,9 @@ class IOActorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach {
"an IO Actor" must {
"run echo server" in {
val ioManager = Actor.actorOf(new IOManager(2)).start // teeny tiny buffer
val server = Actor.actorOf(new SimpleEchoServer("localhost", 8064, ioManager)).start
val client = Actor.actorOf(new SimpleEchoClient("localhost", 8064, ioManager)).start
val ioManager = Actor.actorOf(new IOManager(2)) // teeny tiny buffer
val server = Actor.actorOf(new SimpleEchoServer("localhost", 8064, ioManager))
val client = Actor.actorOf(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")
@ -187,9 +187,9 @@ class IOActorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach {
}
"run echo server under high load" in {
val ioManager = Actor.actorOf(new IOManager()).start
val server = Actor.actorOf(new SimpleEchoServer("localhost", 8065, ioManager)).start
val client = Actor.actorOf(new SimpleEchoClient("localhost", 8065, ioManager)).start
val ioManager = Actor.actorOf(new IOManager())
val server = Actor.actorOf(new SimpleEchoServer("localhost", 8065, ioManager))
val client = Actor.actorOf(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)
@ -199,9 +199,9 @@ class IOActorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach {
}
"run echo server under high load with small buffer" in {
val ioManager = Actor.actorOf(new IOManager(2)).start
val server = Actor.actorOf(new SimpleEchoServer("localhost", 8066, ioManager)).start
val client = Actor.actorOf(new SimpleEchoClient("localhost", 8066, ioManager)).start
val ioManager = Actor.actorOf(new IOManager(2))
val server = Actor.actorOf(new SimpleEchoServer("localhost", 8066, ioManager))
val client = Actor.actorOf(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)
@ -211,10 +211,10 @@ class IOActorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach {
}
"run key-value store" in {
val ioManager = Actor.actorOf(new IOManager(2)).start // teeny tiny buffer
val server = Actor.actorOf(new KVStore("localhost", 8067, ioManager)).start
val client1 = Actor.actorOf(new KVClient("localhost", 8067, ioManager)).start
val client2 = Actor.actorOf(new KVClient("localhost", 8067, ioManager)).start
val ioManager = Actor.actorOf(new IOManager(2)) // teeny tiny buffer
val server = Actor.actorOf(new KVStore("localhost", 8067, ioManager))
val client1 = Actor.actorOf(new KVClient("localhost", 8067, ioManager))
val client2 = Actor.actorOf(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

@ -77,7 +77,7 @@ class LoggingReceiveSpec
def receive = loggable(this) {
case _ self reply "x"
}
}).start()
})
actor ! "buh"
within(1 second) {
expectMsg(EventHandler.Debug(actor.underlyingActor, "received handled message buh"))
@ -104,7 +104,7 @@ class LoggingReceiveSpec
def receive = loggable(this)(loggable(this) {
case _ self reply "x"
})
}).start()
})
actor ! "buh"
within(1 second) {
expectMsg(EventHandler.Debug(actor.underlyingActor, "received handled message buh"))
@ -124,7 +124,7 @@ class LoggingReceiveSpec
def receive = {
case _
}
}).start()
})
actor ! PoisonPill
expectMsg(300 millis, EventHandler.Debug(actor.underlyingActor, "received AutoReceiveMessage PoisonPill"))
}
@ -141,7 +141,7 @@ class LoggingReceiveSpec
def receive = {
case _
}
}).start()
})
val actor1 = actor.underlyingActor
expectMsg(EventHandler.Debug(actor1, "started"))
supervisor link actor

View file

@ -27,7 +27,7 @@ class ReceiveTimeoutSpec extends WordSpec with MustMatchers {
protected def receive = {
case ReceiveTimeout timeoutLatch.open
}
}).start()
})
timeoutLatch.await
timeoutActor.stop()
@ -42,7 +42,7 @@ class ReceiveTimeoutSpec extends WordSpec with MustMatchers {
protected def receive = {
case ReceiveTimeout timeoutLatch.open
}
}).start()
})
timeoutLatch.await
@ -67,7 +67,7 @@ class ReceiveTimeoutSpec extends WordSpec with MustMatchers {
case Tick ()
case ReceiveTimeout timeoutLatch.open
}
}).start()
})
timeoutActor ! Tick
@ -90,7 +90,7 @@ class ReceiveTimeoutSpec extends WordSpec with MustMatchers {
timeoutLatch.open
self.receiveTimeout = None
}
}).start()
})
timeoutActor ! Tick
@ -106,7 +106,7 @@ class ReceiveTimeoutSpec extends WordSpec with MustMatchers {
protected def receive = {
case ReceiveTimeout timeoutLatch.open
}
}).start()
})
timeoutLatch.awaitTimeout(1 second) // timeout expected
timeoutActor.stop()

View file

@ -43,7 +43,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
val countDownLatch = new CountDownLatch(3)
val stopLatch = new StandardLatch
val slave = actorOf(new Actor {
val slave = actorOf(Props(new Actor {
protected def receive = {
case Ping countDownLatch.countDown()
@ -59,8 +59,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
override def postStop() = {
stopLatch.open
}
})
boss.link(slave).start()
}).withSupervisor(boss))
slave ! Ping
slave ! Crash
@ -93,7 +92,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
val countDownLatch = new CountDownLatch(100)
val slave = actorOf(new Actor {
val slave = actorOf(Props(new Actor {
protected def receive = {
case Crash throw new Exception("Crashing...")
@ -102,9 +101,8 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
override def postRestart(reason: Throwable) = {
countDownLatch.countDown()
}
})
}).withSupervisor(boss))
boss.link(slave).start()
(1 to 100) foreach { _ slave ! Crash }
assert(countDownLatch.await(120, TimeUnit.SECONDS))
assert(slave.isRunning)
@ -123,7 +121,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
val pingLatch = new StandardLatch
val secondPingLatch = new StandardLatch
val slave = actorOf(new Actor {
val slave = actorOf(Props(new Actor {
protected def receive = {
case Ping
@ -144,8 +142,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
secondRestartLatch.open
}
}
})
boss.link(slave).start()
}).withSupervisor(boss))
slave ! Ping
slave ! Crash
@ -182,7 +179,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
val countDownLatch = new CountDownLatch(3)
val stopLatch = new StandardLatch
val slave = actorOf(new Actor {
val slave = actorOf(Props(new Actor {
protected def receive = {
case Ping countDownLatch.countDown()
@ -198,8 +195,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
override def postStop() = {
stopLatch.open
}
})
boss.link(slave).start()
}).withSupervisor(boss))
slave ! Ping
slave ! Crash
@ -235,7 +231,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
def receive = { case m: MaximumNumberOfRestartsWithinTimeRangeReached maxNoOfRestartsLatch.open }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))))
val slave = actorOf(new Actor {
val slave = actorOf(Props(new Actor {
protected def receive = {
case Ping countDownLatch.countDown()
@ -249,8 +245,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
override def postStop() = {
stopLatch.open
}
})
boss.link(slave).start()
}).withSupervisor(boss))
slave ! Ping
slave ! Crash

View file

@ -39,7 +39,7 @@ class SchedulerSpec extends JUnitSuite {
val countDownLatch = new CountDownLatch(3)
val tickActor = actorOf(new Actor {
def receive = { case Tick countDownLatch.countDown() }
}).start()
})
// run every 50 millisec
collectFuture(Scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS))
@ -60,7 +60,7 @@ class SchedulerSpec extends JUnitSuite {
val countDownLatch = new CountDownLatch(3)
val tickActor = actorOf(new Actor {
def receive = { case Tick countDownLatch.countDown() }
}).start()
})
// run every 50 millisec
collectFuture(Scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
collectFuture(Scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
@ -80,7 +80,7 @@ class SchedulerSpec extends JUnitSuite {
val ticks = new CountDownLatch(1000)
val actor = actorOf(new Actor {
def receive = { case Ping ticks.countDown }
}).start
})
val numActors = Actor.registry.local.actors.length
(1 to 1000).foreach(_ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS)))
assert(ticks.await(10, TimeUnit.SECONDS))
@ -97,7 +97,7 @@ class SchedulerSpec extends JUnitSuite {
val actor = actorOf(new Actor {
def receive = { case Ping ticks.countDown() }
}).start()
})
(1 to 10).foreach { i
val future = collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS))
@ -133,7 +133,7 @@ class SchedulerSpec extends JUnitSuite {
Supervise(
actor,
Permanent)
:: Nil)).start
:: Nil))
collectFuture(Scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS))
// appx 2 pings before crash

View file

@ -16,13 +16,9 @@ object SupervisorHierarchySpec {
class FireWorkerException(msg: String) extends Exception(msg)
class CountDownActor(countDown: CountDownLatch) extends Actor {
protected def receive = { case _ () }
protected def receive = { case _ }
override def postRestart(reason: Throwable) = countDown.countDown()
}
class CrasherActor extends Actor {
protected def receive = { case _ () }
}
}
class SupervisorHierarchySpec extends JUnitSuite {
@ -32,20 +28,13 @@ class SupervisorHierarchySpec extends JUnitSuite {
def killWorkerShouldRestartMangerAndOtherWorkers = {
val countDown = new CountDownLatch(4)
val workerOne = actorOf(new CountDownActor(countDown))
val workerTwo = actorOf(new CountDownActor(countDown))
val workerThree = actorOf(new CountDownActor(countDown))
val boss = actorOf(Props(new Actor {
protected def receive = { case _ () }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 1000)))
val manager = actorOf(new CountDownActor(countDown))
boss.link(manager).start()
val manager = actorOf(Props(new CountDownActor(countDown)).withSupervisor(boss))
manager.link(workerOne).start()
manager.link(workerTwo).start()
manager.link(workerThree).start()
val workerOne, workerTwo, workerThree = actorOf(Props(new CountDownActor(countDown)).withSupervisor(manager))
workerOne ! Death(workerOne, new FireWorkerException("Fire the worker!"))
@ -57,19 +46,21 @@ class SupervisorHierarchySpec extends JUnitSuite {
@Test
def supervisorShouldReceiveNotificationMessageWhenMaximumNumberOfRestartsWithinTimeRangeIsReached = {
val countDown = new CountDownLatch(2)
val crasher = actorOf(new CountDownActor(countDown))
val countDownMessages = new CountDownLatch(1)
val countDownMax = new CountDownLatch(1)
val boss = actorOf(Props(new Actor {
protected def receive = {
case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) countDown.countDown()
case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) countDownMax.countDown()
}
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 1, 5000)))
boss.link(crasher).start()
crasher ! Death(crasher, new FireWorkerException("Fire the worker!"))
crasher ! Death(crasher, new FireWorkerException("Fire the worker!"))
val crasher = actorOf(Props(new CountDownActor(countDownMessages)).withSupervisor(boss))
assert(countDown.await(2, TimeUnit.SECONDS))
crasher ! Kill
crasher ! Kill
assert(countDownMessages.await(2, TimeUnit.SECONDS))
assert(countDownMax.await(2, TimeUnit.SECONDS))
}
}

View file

@ -57,11 +57,7 @@ object SupervisorSpec {
class Master extends Actor {
val temp = {
val a = actorOf(Props[PingPongActor].withLifeCycle(Temporary))
self link a
a.start
}
val temp = actorOf(Props[PingPongActor].withLifeCycle(Temporary).withSupervisor(self))
override def receive = {
case Die (temp.?(Die, TimeoutMillis)).get
@ -88,7 +84,7 @@ object SupervisorSpec {
}
def singleActorAllForOne = {
val pingpong = actorOf[PingPongActor].start()
val pingpong = actorOf[PingPongActor]
val supervisor = Supervisor(
SupervisorConfig(
@ -102,7 +98,7 @@ object SupervisorSpec {
}
def singleActorOneForOne = {
val pingpong = actorOf[PingPongActor].start()
val pingpong = actorOf[PingPongActor]
val supervisor = Supervisor(
SupervisorConfig(
@ -116,9 +112,9 @@ object SupervisorSpec {
}
def multipleActorsAllForOne = {
val pingpong1 = actorOf[PingPongActor].start()
val pingpong2 = actorOf[PingPongActor].start()
val pingpong3 = actorOf[PingPongActor].start()
val pingpong1 = actorOf[PingPongActor]
val pingpong2 = actorOf[PingPongActor]
val pingpong3 = actorOf[PingPongActor]
val supervisor = Supervisor(
SupervisorConfig(
@ -140,9 +136,9 @@ object SupervisorSpec {
}
def multipleActorsOneForOne = {
val pingpong1 = actorOf[PingPongActor].start()
val pingpong2 = actorOf[PingPongActor].start()
val pingpong3 = actorOf[PingPongActor].start()
val pingpong1 = actorOf[PingPongActor]
val pingpong2 = actorOf[PingPongActor]
val pingpong3 = actorOf[PingPongActor]
val supervisor = Supervisor(
SupervisorConfig(

View file

@ -18,14 +18,14 @@ class Ticket669Spec extends WordSpec with MustMatchers with BeforeAndAfterAll {
override def afterAll = {
Actor.registry.local.shutdownAll
akka.event.EventHandler.start
akka.event.EventHandler.start()
}
"A supervised actor with lifecycle PERMANENT" should {
"be able to reply on failure during preRestart" in {
filterEvents(EventFilter[Exception]("test")) {
val latch = new CountDownLatch(1)
val sender = Actor.actorOf(new Sender(latch)).start()
val sender = Actor.actorOf(new Sender(latch))
val supervised = Actor.actorOf[Supervised]
val supervisor = Supervisor(SupervisorConfig(
@ -40,7 +40,7 @@ class Ticket669Spec extends WordSpec with MustMatchers with BeforeAndAfterAll {
"be able to reply on failure during postStop" in {
filterEvents(EventFilter[Exception]("test")) {
val latch = new CountDownLatch(1)
val sender = Actor.actorOf(new Sender(latch)).start()
val sender = Actor.actorOf(new Sender(latch))
val supervised = Actor.actorOf[Supervised]
val supervisor = Supervisor(SupervisorConfig(

View file

@ -221,7 +221,6 @@ abstract class ActorModelSpec extends JUnitSuite {
implicit val dispatcher = newInterceptedDispatcher
assertDispatcher(dispatcher)(starts = 0, stops = 0)
val a = newTestActor
a.start()
assertDispatcher(dispatcher)(starts = 1, stops = 0)
a.stop()
await(dispatcher.stops.get == 1)(withinMs = dispatcher.timeoutMs * 5)
@ -242,10 +241,7 @@ abstract class ActorModelSpec extends JUnitSuite {
assertDispatcher(dispatcher)(starts = 2, stops = 2)
val a2 = newTestActor
a2.start
val futures2 = for (i 1 to 10) yield Future {
i
}
val futures2 = for (i 1 to 10) yield Future { i }
await(dispatcher.starts.get == 3)(withinMs = dispatcher.timeoutMs * 5)
assertDispatcher(dispatcher)(starts = 3, stops = 2)
@ -258,9 +254,8 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldProcessMessagesOneAtATime {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor
val start, oneAtATime = new CountDownLatch(1)
a.start()
val a = newTestActor
a ! CountDown(start)
assertCountDown(start, Testing.testTime(3000), "Should process first message within 3 seconds")
@ -279,9 +274,8 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldHandleQueueingFromMultipleThreads {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor
val counter = new CountDownLatch(200)
a.start()
val a = newTestActor
for (i 1 to 10) {
spawn {
@ -312,8 +306,8 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldProcessMessagesInParallel: Unit = {
implicit val dispatcher = newInterceptedDispatcher
val a, b = newTestActor.start()
val aStart, aStop, bParallel = new CountDownLatch(1)
val a, b = newTestActor
a ! Meet(aStart, aStop)
assertCountDown(aStart, Testing.testTime(3000), "Should process first message within 3 seconds")
@ -332,7 +326,7 @@ abstract class ActorModelSpec extends JUnitSuite {
def dispatcherShouldSuspendAndResumeAFailingNonSupervisedPermanentActor {
filterEvents(EventFilter[Exception]("Restart")) {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start()
val a = newTestActor
val done = new CountDownLatch(1)
a ! Restart
a ! CountDown(done)
@ -346,7 +340,7 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldNotProcessMessagesForASuspendedActor {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start().asInstanceOf[LocalActorRef]
val a = newTestActor.asInstanceOf[LocalActorRef]
val done = new CountDownLatch(1)
dispatcher.suspend(a)
a ! CountDown(done)
@ -370,7 +364,7 @@ abstract class ActorModelSpec extends JUnitSuite {
def flood(num: Int) {
val cachedMessage = CountDownNStop(new CountDownLatch(num))
(1 to num) foreach { _
newTestActor.start() ! cachedMessage
newTestActor ! cachedMessage
}
assertCountDown(cachedMessage.latch, Testing.testTime(10000), "Should process " + num + " countdowns")
}
@ -384,7 +378,7 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldCompleteAllUncompletedSenderFuturesOnDeregister {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start().asInstanceOf[LocalActorRef]
val a = newTestActor.asInstanceOf[LocalActorRef]
dispatcher.suspend(a)
val f1: Future[String] = a ? Reply("foo") mapTo manifest[String]
val stopped = a ? PoisonPill
@ -401,7 +395,7 @@ abstract class ActorModelSpec extends JUnitSuite {
def dispatcherShouldContinueToProcessMessagesWhenAThreadGetsInterrupted {
filterEvents(EventFilter[InterruptedException]("Ping!"), EventFilter[akka.event.EventHandler.EventHandlerException]) {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start()
val a = newTestActor
val f1 = a ? Reply("foo")
val f2 = a ? Reply("bar")
val f3 = a ? Interrupt
@ -426,7 +420,7 @@ abstract class ActorModelSpec extends JUnitSuite {
def dispatcherShouldContinueToProcessMessagesWhenExceptionIsThrown {
filterEvents(EventFilter[IndexOutOfBoundsException], EventFilter[RemoteException]) {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start()
val a = newTestActor
val f1 = a ? Reply("foo")
val f2 = a ? Reply("bar")
val f3 = a ? new ThrowException(new IndexOutOfBoundsException("IndexOutOfBoundsException"))

View file

@ -35,8 +35,8 @@ class DispatcherActorsSpec extends JUnitSuite with MustMatchers {
def slowActorShouldntBlockFastActor {
val sFinished = new CountDownLatch(50)
val fFinished = new CountDownLatch(10)
val s = actorOf(new SlowActor(sFinished), "SlowActor").start
val f = actorOf(new FastActor(fFinished), "FastActor").start
val s = actorOf(new SlowActor(sFinished), "SlowActor")
val f = actorOf(new FastActor(fFinished), "FastActor")
// send a lot of stuff to s
for (i 1 to 50) {

View file

@ -22,7 +22,7 @@ class ListenerSpec extends WordSpec with MustMatchers {
def receive = listenerManagement orElse {
case "foo" gossip("bar")
}
}).start()
})
def newListener = actorOf(new Actor {
def receive = {
@ -32,7 +32,7 @@ class ListenerSpec extends WordSpec with MustMatchers {
case "foo"
fooLatch.countDown()
}
}).start()
})
val a1 = newListener
val a2 = newListener

View file

@ -120,7 +120,6 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
"returns a result" must {
behave like futureWithResult { test
val actor = actorOf[TestActor]
actor.start()
val future = actor ? "Hello"
future.await
test(future, "World")
@ -131,7 +130,6 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
behave like futureWithException[RuntimeException] { test
filterException[RuntimeException] {
val actor = actorOf[TestActor]
actor.start()
val future = actor ? "Failure"
future.await
test(future, "Expected exception; to test fault-tolerance")
@ -144,8 +142,8 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
"using flatMap with an Actor" that {
"will return a result" must {
behave like futureWithResult { test
val actor1 = actorOf[TestActor].start()
val actor2 = actorOf(new Actor { def receive = { case s: String self reply s.toUpperCase } }).start()
val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String self reply s.toUpperCase } })
val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s }
future.await
test(future, "WORLD")
@ -156,8 +154,8 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
"will throw an exception" must {
behave like futureWithException[ArithmeticException] { test
filterException[ArithmeticException] {
val actor1 = actorOf[TestActor].start()
val actor2 = actorOf(new Actor { def receive = { case s: String self reply (s.length / 0) } }).start()
val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String self reply (s.length / 0) } })
val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s }
future.await
test(future, "/ by zero")
@ -169,8 +167,8 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
"will throw a MatchError when matching wrong type" must {
behave like futureWithException[MatchError] { test
filterException[MatchError] {
val actor1 = actorOf[TestActor].start()
val actor2 = actorOf(new Actor { def receive = { case s: String self reply s.toUpperCase } }).start()
val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String self reply s.toUpperCase } })
val future = actor1 ? "Hello" flatMap { case i: Int actor2 ? i }
future.await
test(future, "World (of class java.lang.String)")
@ -190,7 +188,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
case s: String self reply s.length
case i: Int self reply (i * 2).toString
}
}).start()
})
val future0 = actor ? "Hello"
@ -222,7 +220,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
case Req(s: String) self reply Res(s.length)
case Req(i: Int) self reply Res((i * 2).toString)
}
}).start()
})
val future1 = for {
Res(a: Int) actor ? Req("Hello")
@ -262,7 +260,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
val future7 = future3 recover { case e: ArithmeticException "You got ERROR" }
val actor = actorOf[TestActor].start()
val actor = actorOf[TestActor]
val future8 = actor ? "Failure"
val future9 = actor ? "Failure" recover {
@ -307,7 +305,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
val actors = (1 to 10).toList map { _
actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); self tryReply add }
}).start()
})
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200), timeout).mapTo[Int] }
@ -318,7 +316,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
val actors = (1 to 10).toList map { _
actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); self tryReply add }
}).start()
})
}
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)
@ -334,7 +332,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected")
self tryReply add
}
}).start()
})
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 100), timeout).mapTo[Int] }
@ -365,7 +363,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
val actors = (1 to 10).toList map { _
actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); self tryReply add }
}).start()
})
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200), timeout).mapTo[Int] }
@ -382,7 +380,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected")
self tryReply add
}
}).start()
})
}
val timeout = 10000
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 100), timeout).mapTo[Int] }
@ -398,7 +396,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
"receiveShouldExecuteOnComplete" in {
val latch = new StandardLatch
val actor = actorOf[TestActor].start()
val actor = actorOf[TestActor]
actor ? "Hello" onResult { case "World" latch.open }
assert(latch.tryAwait(5, TimeUnit.SECONDS))
actor.stop()
@ -412,7 +410,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
self reply counter
counter += 2
}
}).start()
})
val oddFutures = List.fill(100)(oddActor ? 'GetNext mapTo manifest[Int])
assert(Future.sequence(oddFutures).get.sum === 10000)
@ -472,7 +470,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
"futureComposingWithContinuations" in {
import Future.flow
val actor = actorOf[TestActor].start
val actor = actorOf[TestActor]
val x = Future("Hello")
val y = x flatMap (actor ? _) mapTo manifest[String]
@ -501,7 +499,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
filterException[ClassCastException] {
import Future.flow
val actor = actorOf[TestActor].start
val actor = actorOf[TestActor]
val x = Future(3)
val y = (actor ? "Hello").mapTo[Int]
@ -516,7 +514,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
filterException[ClassCastException] {
import Future.flow
val actor = actorOf[TestActor].start
val actor = actorOf[TestActor]
val x = Future("Hello")
val y = actor ? "Hello" mapTo manifest[Nothing]

View file

@ -76,7 +76,7 @@ abstract class MailboxSpec extends WordSpec with MustMatchers with BeforeAndAfte
case e: Throwable result.completeWithException(e)
}
})
t.start()
t.start
result
}

View file

@ -34,13 +34,12 @@ abstract class AkkaPerformanceTest extends BenchmarkScenarios {
val oddRepeats = repeat - (repeatsPerClient * numberOfClients)
val latch = new CountDownLatch(numberOfClients)
val receivers = tradingSystem.orderReceivers.toIndexedSeq
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
actorOf(Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs)).withDispatcher(clientDispatcher))
}).toList
Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs)).withDispatcher(clientDispatcher)
}).toList.map(actorOf(_))
clients.foreach(_.start)
val start = System.nanoTime
clients.foreach(_ ! "run")
val ok = latch.await((5000 + (2 + delayMs) * totalNumberOfRequests) * timeDilation, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)

View file

@ -96,14 +96,11 @@ class AkkaTradingSystem extends TradingSystem {
override def start() {
for (MatchingEngineInfo(p, s, o) matchingEngines) {
p.start()
// standby is optional
s.foreach(_.start())
s.foreach(p ! _)
}
val routing = matchingEngineRouting
for (or orderReceivers) {
or.start()
or ! routing
}
}

View file

@ -45,7 +45,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
latch.countDown()
self tryReply "success"
}
}).start()
})
def limit = 2
def selectionCount = 1
@ -59,7 +59,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def receive = {
case "success" successes.countDown()
}
}).start()
})
implicit val replyTo = successCounter
pool ! "a"
@ -387,7 +387,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
throw new RuntimeException
case _ pingCount.incrementAndGet
}
}).start()
})
}).withFaultHandler(faultHandler))
val pool2 = actorOf(
@ -411,7 +411,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
throw new RuntimeException
case _ pingCount.incrementAndGet
}
}).start()
})
}).withFaultHandler(faultHandler))
val pool3 = actorOf(
@ -534,7 +534,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
throw new RuntimeException
case _ pingCount.incrementAndGet
}
}).start()
})
}).withFaultHandler(OneForOneStrategy(List(classOf[IllegalStateException]), 5, 1000)))
// actor comes back right away

View file

@ -26,7 +26,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
"direct router" must {
"be started when constructed" in {
val actor1 = Actor.actorOf[TestActor].start
val actor1 = Actor.actorOf[TestActor]
val actor = Routing.actorOf("foo", List(actor1), RouterType.Direct)
actor.isRunning must be(true)
@ -50,7 +50,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case _ counter.incrementAndGet
}
}).start()
})
val routedActor = Routing.actorOf("foo", List(connection1), RouterType.Direct)
routedActor ! "hello"
@ -70,7 +70,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
}).start()
})
val actor = Routing.actorOf("foo", List(connection1), RouterType.Direct)
@ -86,7 +86,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
"round robin router" must {
"be started when constructed" in {
val actor1 = Actor.actorOf[TestActor].start
val actor1 = Actor.actorOf[TestActor]
val actor = Routing.actorOf("foo", List(actor1), RouterType.RoundRobin)
actor.isRunning must be(true)
@ -121,7 +121,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg)
}
}).start()
})
connections = connections :+ connection
}
@ -154,7 +154,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
}).start()
})
val counter2 = new AtomicInteger
val connection2 = actorOf(new Actor {
@ -162,7 +162,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
}).start()
})
val actor = Routing.actorOf("foo", List(connection1, connection2), RouterType.RoundRobin)
@ -184,7 +184,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case _ counter1.incrementAndGet()
}
}).start()
})
val actor = Routing.actorOf("foo", List(connection1), RouterType.RoundRobin)
@ -205,7 +205,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
"be started when constructed" in {
val actor1 = Actor.actorOf[TestActor].start
val actor1 = Actor.actorOf[TestActor]
val actor = Routing.actorOf("foo", List(actor1), RouterType.Random)
actor.isRunning must be(true)
@ -233,7 +233,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
}).start()
})
val counter2 = new AtomicInteger
val connection2 = actorOf(new Actor {
@ -241,7 +241,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
}).start()
})
val actor = Routing.actorOf("foo", List(connection1, connection2), RouterType.Random)
@ -263,7 +263,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case _ counter1.incrementAndGet()
}
}).start()
})
val actor = Routing.actorOf("foo", List(connection1), RouterType.Random)
@ -282,7 +282,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
"least cpu router" must {
"throw IllegalArgumentException when constructed" in {
val actor1 = Actor.actorOf[TestActor].start
val actor1 = Actor.actorOf[TestActor]
try {
Routing.actorOf("foo", List(actor1), RouterType.LeastCPU)
@ -294,7 +294,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
"least ram router" must {
"throw IllegalArgumentException when constructed" in {
val actor1 = Actor.actorOf[TestActor].start
val actor1 = Actor.actorOf[TestActor]
try {
Routing.actorOf("foo", List(actor1), RouterType.LeastRAM)
@ -306,7 +306,7 @@ class RoutingSpec extends WordSpec with MustMatchers {
"smallest mailbox" must {
"throw IllegalArgumentException when constructed" in {
val actor1 = Actor.actorOf[TestActor].start
val actor1 = Actor.actorOf[TestActor]
try {
Routing.actorOf("foo", List(actor1), RouterType.LeastMessages)

View file

@ -25,9 +25,9 @@ class CallingThreadDispatcherModelSpec extends ActorModelSpec {
def flood(num: Int) {
val cachedMessage = CountDownNStop(new CountDownLatch(num))
val keeper = newTestActor.start()
val keeper = newTestActor
(1 to num) foreach { _
newTestActor.start() ! cachedMessage
newTestActor ! cachedMessage
}
keeper.stop()
assertCountDown(cachedMessage.latch, 10000, "Should process " + num + " countdowns")

View file

@ -18,7 +18,7 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
val shutdownLatch = new CountDownLatch(1)
val props = RoutedProps.apply()
val props = RoutedProps()
.withDeployId("foo")
.withConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
@ -118,7 +118,7 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg)
}
}).start()
})
connections = connections :+ connection
}
@ -154,7 +154,7 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
}).start()
})
val counter2 = new AtomicInteger
val connection2 = actorOf(new Actor {
@ -162,7 +162,7 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
}).start()
})
val props = RoutedProps.apply()
.withDeployId("foo")
@ -189,7 +189,7 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
case _id: Int if (_id == id)
case _ Thread sleep 100 * id; self tryReply id
}
}).start()
})
}

View file

@ -45,7 +45,6 @@ public class Actors {
* return new MyUntypedActor("service:name", 5);
* }
* }, "my-actor-address");
* actor.start();
* actor.tell(message, context);
* actor.stop();
* </pre>
@ -69,7 +68,6 @@ public class Actors {
* return new MyUntypedActor("service:name", 5);
* }
* });
* actor.start();
* actor.tell(message, context);
* actor.stop();
* </pre>
@ -83,13 +81,12 @@ public class Actors {
* Example in Java:
* <pre>
* ActorRef actor = Actors.actorOf(MyUntypedActor.class, "my-actor-address");
* actor.start();
* actor.tell(message, context);
* actor.stop();
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = Actors.actorOf(MyActor.class, "my-actor-address").start();
* val actor = Actors.actorOf(MyActor.class, "my-actor-address");
* </pre>
*/
public static ActorRef actorOf(final Class<? extends Actor> type, final String address) {
@ -101,13 +98,12 @@ public class Actors {
* Example in Java:
* <pre>
* ActorRef actor = Actors.actorOf(MyUntypedActor.class, "my-actor-address");
* actor.start();
* actor.tell(message, context);
* actor.stop();
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = Actors.actorOf(MyActor.class).start();
* val actor = Actors.actorOf(MyActor.class);
* </pre>
*/
public static ActorRef actorOf(final Class<? extends Actor> type) {

View file

@ -240,13 +240,12 @@ object Actor {
* <pre>
* import Actor._
* val actor = actorOf[MyActor]
* actor.start()
* actor ! message
* actor.stop()
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf[MyActor].start()
* val actor = actorOf[MyActor]
* </pre>
*/
def actorOf[T <: Actor: Manifest](address: String): ActorRef =
@ -258,13 +257,12 @@ object Actor {
* <pre>
* import Actor._
* val actor = actorOf[MyActor]
* actor.start
* actor ! message
* actor.stop
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf[MyActor].start
* val actor = actorOf[MyActor]
* </pre>
*/
def actorOf[T <: Actor: Manifest]: ActorRef =
@ -276,13 +274,12 @@ object Actor {
* <pre>
* import Actor._
* val actor = actorOf(classOf[MyActor])
* actor.start()
* actor ! message
* actor.stop()
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(classOf[MyActor]).start()
* val actor = actorOf(classOf[MyActor])
* </pre>
*/
def actorOf[T <: Actor](clazz: Class[T]): ActorRef =
@ -293,18 +290,15 @@ object Actor {
* <pre>
* import Actor._
* val actor = actorOf(classOf[MyActor])
* actor.start
* actor ! message
* actor.stop
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(classOf[MyActor]).start
* val actor = actorOf(classOf[MyActor])
* </pre>
*/
def actorOf[T <: Actor](clazz: Class[T], address: String): ActorRef = {
createActor(address, () newLocalActorRef(clazz, address))
}
def actorOf[T <: Actor](clazz: Class[T], address: String): ActorRef = actorOf(Props(clazz), address)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory function
@ -315,16 +309,15 @@ object Actor {
* <pre>
* import Actor._
* val actor = actorOf(new MyActor)
* actor.start()
* actor ! message
* actor.stop()
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(new MyActor).start()
* val actor = actorOf(new MyActor)
* </pre>
*/
def actorOf[T <: Actor](factory: T): ActorRef = actorOf(factory, new UUID().toString)
def actorOf[T <: Actor](factory: T): ActorRef = actorOf(factory, newUuid().toString)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory function
@ -335,18 +328,15 @@ object Actor {
* <pre>
* import Actor._
* val actor = actorOf(new MyActor)
* actor.start
* actor ! message
* actor.stop
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(new MyActor).start
* val actor = actorOf(new MyActor)
* </pre>
*/
def actorOf[T <: Actor](creator: T, address: String): ActorRef = {
createActor(address, () new LocalActorRef(Props(creator = () creator), address))
}
def actorOf[T <: Actor](creator: T, address: String): ActorRef = actorOf(Props(creator), address)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory (Creator<Actor>)
@ -356,8 +346,7 @@ object Actor {
* <p/>
* JAVA API
*/
def actorOf[T <: Actor](creator: Creator[T]): ActorRef =
actorOf(creator, newUuid().toString)
def actorOf[T <: Actor](creator: Creator[T]): ActorRef = actorOf(Props(creator), newUuid().toString)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory (Creator<Actor>)
@ -367,25 +356,62 @@ object Actor {
* This function should <b>NOT</b> be used for remote actors.
* JAVA API
*/
def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = {
createActor(address, () new LocalActorRef(Props(creator = () creator.create), address))
}
def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = actorOf(Props(creator), address)
def actorOf(props: Props): ActorRef = actorOf(props, newUuid.toString)
//TODO FIXME
def actorOf(props: Props, address: String): ActorRef = {
//TODO Implement support for configuring by deployment ID etc
//TODO If localOnly = true, never use the config file deployment and always create a new actor
//TODO If deployId matches an already created actor (Ahead-of-time deployed) return that actor
//TODO If deployId exists in config, it will override the specified Props (should we attempt to merge?)
/*val address = props.deployId match { //TODO handle deployId separately from address?
case "" | null newUuid().toString
//FIXME Hack, either remove deployId or rename it to address and remove the other param
val realAddress = props.deployId match { //TODO handle deployId separately from address?
case Props.`defaultDeployId` | null address
case other other
} */
new LocalActorRef(props, address).start()
}
createActor(realAddress, props, systemService = false)
}
private[akka] def createActor(address: String, props: Props, systemService: Boolean): ActorRef = if (!systemService) {
Address.validate(address)
registry.actorFor(address) match { // check if the actor for the address is already in the registry
case Some(actorRef) actorRef // it is -> return it
case None // it is not -> create it
try {
Deployer.deploymentFor(address) match {
case Deploy(_, _, router, _, Local) new LocalActorRef(props, address, systemService) // create a local actor
case deploy
val factory = props.creator
newClusterActorRef(() new LocalActorRef(Props(factory), address, systemService), address, deploy) //create a clustered actor
}
} catch {
case e: DeploymentException
EventHandler.error(e, this, "Look up deployment for address [%s] falling back to local actor." format address)
new LocalActorRef(props, address, systemService) // if deployment fails, fall back to local actors
}
}
} else new LocalActorRef(props, address, systemService)
private[akka] def createActor(address: String, factory: () ActorRef, systemService: Boolean): ActorRef = if (!systemService) {
Address.validate(address)
registry.actorFor(address) match { // check if the actor for the address is already in the registry
case Some(actorRef) actorRef // it is -> return it
case None // it is not -> create it
try {
Deployer.deploymentFor(address) match {
case Deploy(_, _, router, _, Local) factory() // create a local actor
case deploy newClusterActorRef(factory, address, deploy) //create a clustered actor
}
} catch {
case e: DeploymentException
EventHandler.error(e, this, "Look up deployment for address [%s] falling back to local actor." format address)
factory() // if deployment fails, fall back to local actors
}
}
} else factory()
/**
* Use to spawn out a block of code in an event-driven actor. Will shut actor down when
* the block has been executed.
@ -409,48 +435,6 @@ object Actor {
}).withDispatcher(dispatcher)) ! "go"
}
/**
* Creates an actor according to the deployment plan for the 'address'; local or clustered.
* If already created then it just returns it from the registry.
*/
private[akka] def createActor(address: String, actorFactory: () ActorRef): ActorRef = {
Address.validate(address)
registry.actorFor(address) match { // check if the actor for the address is already in the registry
case Some(actorRef) actorRef // it is -> return it
case None // it is not -> create it
try {
Deployer.deploymentFor(address) match {
case Deploy(_, _, router, _, Local) actorFactory() // create a local actor
case deploy newClusterActorRef(actorFactory, address, deploy)
}
} catch {
case e: DeploymentException
EventHandler.error(e, this, "Look up deployment for address [%s] falling back to local actor." format address)
actorFactory() // if deployment fails, fall back to local actors
}
}
}
private[akka] def newLocalActorRef(clazz: Class[_ <: Actor], address: String): ActorRef = {
new LocalActorRef(Props(creator = () {
import ReflectiveAccess.{ createInstance, noParams, noArgs }
createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs) match {
case Right(actor) actor
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
throw new ActorInitializationException(
"Could not instantiate Actor of " + clazz +
"\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)'.", cause)
}
}), address)
}
private[akka] def newClusterActorRef(factory: () ActorRef, address: String, deploy: Deploy): ActorRef =
deploy match {
case Deploy(configAddress, recipe, router, failureDetector, Clustered(preferredHomeNodes, replicas, replication))
@ -601,7 +585,6 @@ trait Actor {
* For example fields like:
* <pre>
* self.dispatcher = ...
* self.trapExit = ...
* self.faultHandler = ...
* self.lifeCycle = ...
* self.sender
@ -612,7 +595,6 @@ trait Actor {
* self.reply(..)
* self.link(..)
* self.unlink(..)
* self.start(..)
* self.stop(..)
* </pre>
*/
@ -645,7 +627,7 @@ trait Actor {
/**
* User overridable callback.
* <p/>
* Is called when an Actor is started by invoking 'actor.start()'.
* Is called when an Actor is started by invoking 'actor'.
*/
def preStart() {}

View file

@ -53,7 +53,6 @@ object Props {
final val defaultLifeCycle: LifeCycle = Permanent
final val defaultFaultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy
final val defaultSupervisor: Option[ActorRef] = None
final val defaultLocalOnly: Boolean = false
/**
* The default Props instance, uses the settings from the Props object starting with default*
@ -85,8 +84,13 @@ object Props {
*/
def apply(creator: Actor): Props = default.withCreator(creator)
def apply(behavior: (ScalaActorRef with SelfActorRef) Actor.Receive): Props =
apply(new Actor { def receive = behavior(self) })
/**
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
* using the supplied thunk
*/
def apply(creator: Creator[_ <: Actor]): Props = default.withCreator(creator.create)
def apply(behavior: (ScalaActorRef with SelfActorRef) Actor.Receive): Props = apply(new Actor { def receive = behavior(self) })
}
/**
@ -94,12 +98,11 @@ object Props {
*/
case class Props(creator: () Actor = Props.defaultCreator,
deployId: String = Props.defaultDeployId,
dispatcher: MessageDispatcher = Props.defaultDispatcher,
@transient dispatcher: MessageDispatcher = Props.defaultDispatcher,
timeout: Timeout = Props.defaultTimeout,
lifeCycle: LifeCycle = Props.defaultLifeCycle,
faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler,
supervisor: Option[ActorRef] = Props.defaultSupervisor,
localOnly: Boolean = Props.defaultLocalOnly) {
supervisor: Option[ActorRef] = Props.defaultSupervisor) {
/**
* No-args constructor that sets all the default values
* Java API
@ -111,8 +114,7 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
timeout = Props.defaultTimeout,
lifeCycle = Props.defaultLifeCycle,
faultHandler = Props.defaultFaultHandler,
supervisor = Props.defaultSupervisor,
localOnly = Props.defaultLocalOnly)
supervisor = Props.defaultSupervisor)
/**
* Returns a new Props with the specified creator set
@ -173,12 +175,6 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
* Scala API
*/
def withSupervisor(s: scala.Option[ActorRef]) = copy(supervisor = s)
/**
* Returns a new Props with the specified localOnly set
* Java and Scala API
*/
def withLocalOnly(l: Boolean) = copy(localOnly = l)
}
/**
@ -191,14 +187,13 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
* import Actor._
*
* val actor = actorOf[MyActor]
* actor.start()
* actor ! message
* actor.stop()
* </pre>
*
* You can also create and start actors like this:
* <pre>
* val actor = actorOf[MyActor].start()
* val actor = actorOf[MyActor]
* </pre>
*
* Here is an example on how to create an actor with a non-default constructor.
@ -206,7 +201,6 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
* import Actor._
*
* val actor = actorOf(new MyActor(...))
* actor.start()
* actor ! message
* actor.stop()
* </pre>
@ -218,6 +212,7 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha
// Only mutable for RemoteServer in order to maintain identity across nodes
@volatile
protected[akka] var _uuid = newUuid
@volatile
protected[this] var _status: ActorRefInternals.StatusType = ActorRefInternals.UNSTARTED
@ -241,19 +236,6 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha
def uuid = _uuid
/**
* Is the actor running?
*/
def isRunning: Boolean = _status match {
case ActorRefInternals.BEING_RESTARTED | ActorRefInternals.RUNNING true
case _ false
}
/**
* Is the actor shut down?
*/
def isShutdown: Boolean = _status == ActorRefInternals.SHUTDOWN
/**
* Only for internal use. UUID is effectively final.
*/
@ -312,11 +294,6 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha
else forward(message)(sender)
}
/**
* Starts up the actor and its message queue.
*/
def start(): this.type
/**
* Shuts down the actor its dispatcher and message queue.
* Alias for 'stop'.
@ -328,7 +305,20 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha
/**
* Shuts down the actor its dispatcher and message queue.
*/
def stop()
def stop(): Unit
/**
* Is the actor running?
*/
def isRunning: Boolean = _status match { //TODO Remove this method
case ActorRefInternals.BEING_RESTARTED | ActorRefInternals.RUNNING true
case _ false
}
/**
* Is the actor shut down?
*/
def isShutdown: Boolean = _status == ActorRefInternals.SHUTDOWN
/**
* Links an other actor to this actor. Links are unidirectional and means that a the linking actor will
@ -511,10 +501,10 @@ abstract class SelfActorRef extends ActorRef with ForwardableChannel { self: Loc
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class LocalActorRef private[akka] (private[this] val props: Props, val address: String)
class LocalActorRef private[akka] (private[this] val props: Props, val address: String, val systemService: Boolean)
extends SelfActorRef with ScalaActorRef {
protected[akka] val guard = new ReentrantGuard
protected[akka] val guard = new ReentrantGuard //TODO FIXME remove the last synchronization point
@volatile
protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@ -550,12 +540,12 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
"Could not create Serializer object for [" + this.getClass.getName + "]")
}
private lazy val hasReplicationStorage: Boolean = {
private lazy val hasReplicationStorage: Boolean = if (!systemService) {
import DeploymentConfig._
isReplicated(replicationSchemeFor(Deployer.deploymentFor(address)).getOrElse(Transient))
}
} else false
private lazy val replicationStorage: Option[TransactionLog] = {
private lazy val replicationStorage: Option[TransactionLog] = if (!systemService) {
import DeploymentConfig._
val replicationScheme = replicationSchemeFor(Deployer.deploymentFor(address)).getOrElse(Transient)
if (isReplicated(replicationScheme)) {
@ -569,7 +559,7 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
throw new ConfigurationException("Unknown replication storage type [" + replicationScheme + "]")
}
} else None
}
} else None
// If it was started inside "newActor", initialize it
if (isRunning) initializeActorInstance
@ -582,13 +572,12 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
__receiveTimeout: Option[Long],
__hotswap: Stack[PartialFunction[Any, Unit]]) = {
this(__props, __address)
this(__props, __address, systemService = false) //Doesn't make any sense to move a system service
_uuid = __uuid
hotswap = __hotswap
receiveTimeout = __receiveTimeout
setActorSelfFields(actorInstance.get(), this) //TODO Why is this needed?
start()
}
// ========= PUBLIC FUNCTIONS =========
@ -601,9 +590,8 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
/**
* Starts up the actor and its message queue.
*/
def start(): this.type = guard.withGuard[this.type] {
if (isShutdown) throw new ActorStartException(
"Can't restart an actor that has been shut down with 'stop' or 'exit'")
protected[akka] def startInternal(): this.type = guard.withGuard[this.type] {
if (isShutdown) throw new ActorStartException("Can't restart an actor that has been shut down with 'stop' or 'exit'")
if (!isRunning) {
dispatcher.attach(this)
_status = ActorRefInternals.RUNNING
@ -642,8 +630,13 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
a.postStop()
} finally {
currentMessage = null
setActorSelfFields(actorInstance.get, null)
try { //When a supervisor is stopped, it's linked actors should also be stopped
val i = _linkedActors.values.iterator
while (i.hasNext) {
i.next.stop()
i.remove
}
} finally { setActorSelfFields(actorInstance.get, null) }
}
}
@ -741,19 +734,20 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
}
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit =
dispatcher dispatchMessage new MessageInvocation(this, message, channel)
if (isRunning) dispatcher dispatchMessage new MessageInvocation(this, message, channel)
else throw new ActorInitializationException("Actor has not been started, you need to invoke 'actor' before using it")
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
channel: UntypedChannel): Future[Any] = if (isRunning) {
val future = channel match {
case f: ActorPromise f
case _ new ActorPromise(timeout)(dispatcher)
}
dispatcher dispatchMessage new MessageInvocation(this, message, future)
future
}
} else throw new ActorInitializationException("Actor has not been started, you need to invoke 'actor' before using it")
/**
* Callback for the dispatcher. This is the single entry point to the user Actor implementation.
@ -800,8 +794,7 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
dead.restart(reason, maxRetries, within)
case _
if (_supervisor.isDefined) notifySupervisorWithMessage(Death(this, reason))
else dead.stop()
if (_supervisor.isDefined) throw reason else dead.stop() //Escalate problem if not handled here
}
}
@ -975,22 +968,8 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
}
private def notifySupervisorWithMessage(notification: LifeCycleMessage) {
// FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
_supervisor.foreach { sup
if (sup.isShutdown) {
// if supervisor is shut down, game over for all linked actors
//Scoped stop all linked actors, to avoid leaking the 'i' val
{
val i = _linkedActors.values.iterator
while (i.hasNext) {
i.next.stop()
i.remove
}
}
//Stop the actor itself
stop
} else sup ! notification // else notify supervisor
}
val sup = _supervisor
if (sup.isDefined) sup.get ! notification
}
private def setActorSelfFields(actor: Actor, value: ActorRef) {
@ -1043,6 +1022,8 @@ class LocalActorRef private[akka] (private[this] val props: Props, val address:
_futureTimeout = None
}
}
startInternal()
}
/**
@ -1069,8 +1050,6 @@ private[akka] case class RemoteActorRef private[akka] (
ClusterModule.ensureEnabled()
start()
protected[akka] override def timeout: Long = _timeout
def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = {
@ -1091,12 +1070,6 @@ private[akka] case class RemoteActorRef private[akka] (
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
}
def start(): this.type = synchronized[this.type] {
if (_status == ActorRefInternals.UNSTARTED)
_status = ActorRefInternals.RUNNING
this
}
def stop() {
synchronized {
if (_status == ActorRefInternals.RUNNING) {
@ -1126,6 +1099,10 @@ private[akka] case class RemoteActorRef private[akka] (
}
private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef")
/* If you start me up... */
if (_status == ActorRefInternals.UNSTARTED)
_status = ActorRefInternals.RUNNING
}
/**
@ -1156,8 +1133,7 @@ trait ActorRefShared {
* There are implicit conversions in ../actor/Implicits.scala
* from ActorRef -> ScalaActorRef and back
*/
trait ScalaActorRef extends ActorRefShared with ReplyChannel[Any] {
ref: ActorRef
trait ScalaActorRef extends ActorRefShared with ReplyChannel[Any] { ref: ActorRef
/**
* Sends a one-way asynchronous message. E.g. fire-and-forget semantics.
@ -1173,23 +1149,12 @@ trait ScalaActorRef extends ActorRefShared with ReplyChannel[Any] {
* </pre>
* <p/>
*/
def !(message: Any)(implicit channel: UntypedChannel): Unit = {
if (isRunning) postMessageToMailbox(message, channel)
else throw new ActorInitializationException(
"Actor has not been started, you need to invoke 'actor.start()' before using it")
}
def !(message: Any)(implicit channel: UntypedChannel): Unit = postMessageToMailbox(message, channel)
/**
* Sends a message asynchronously, returning a future which may eventually hold the reply.
*/
def ?(message: Any)(implicit channel: UntypedChannel, timeout: Timeout): Future[Any] = {
//FIXME: so it can happen that a message is posted after the actor has been shut down (the isRunning and postMessageToMailboxAndCreateFutureResultWithTimeout are not atomic.
if (isRunning) {
postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel)
//FIXME: there is no after check if the running state is still true.. so no 'repairing'
} else throw new ActorInitializationException(
"Actor has not been started, you need to invoke 'actor.start()' before using it")
}
def ?(message: Any)(implicit channel: UntypedChannel, timeout: Timeout): Future[Any] = postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel)
def ?(message: Any, timeout: Timeout)(implicit channel: UntypedChannel): Future[Any] = ?(message)(channel, timeout)
@ -1198,14 +1163,7 @@ trait ScalaActorRef extends ActorRefShared with ReplyChannel[Any] {
* <p/>
* Works with '!' and '?'/'ask'.
*/
def forward(message: Any)(implicit channel: ForwardableChannel) = {
//FIXME: so it can happen that a message is posted after the actor has been shut down (the isRunning and postMessageToMailbox are not atomic.
if (isRunning) {
postMessageToMailbox(message, channel.channel)
//FIXME: there is no after check if the running state is still true.. so no 'repairing'
} else throw new ActorInitializationException(
"Actor has not been started, you need to invoke 'actor.start()' before using it")
}
def forward(message: Any)(implicit channel: ForwardableChannel) = postMessageToMailbox(message, channel.channel)
}
/**

View file

@ -287,7 +287,7 @@ object LocalDeployer extends ActorDeployer {
deployments.putIfAbsent(deployment.address, deployment) /* match {
case null
deployment match {
case Deploy(address, Some(recipe), routing, _) Actor.actorOf(recipe.implementationClass, address).start() //FIXME use routing?
case Deploy(address, Some(recipe), routing, _) Actor.actorOf(recipe.implementationClass, address) //FIXME use routing?
case _
}
case `deployment` //Already deployed TODO should it be like this?

View file

@ -258,7 +258,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor {
override def preStart: Unit = {
worker = new IOWorker(self, bufferSize)
worker.start
worker.start()
}
def receive = {
@ -307,7 +307,7 @@ private[akka] class IOWorker(ioManager: ActorRef, val bufferSize: Int) {
addRequest(request)
def start(): Unit =
thread.start
thread.start()
// private

View file

@ -79,7 +79,7 @@ object Supervisor {
*
* <pre>
* val supervisor = factory.newInstance
* supervisor.start // start up all managed servers
* supervisor.start() // start up all managed servers
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -91,8 +91,7 @@ case class SupervisorFactory(val config: SupervisorConfig) {
def newInstanceFor(config: SupervisorConfig): Supervisor = {
val supervisor = new Supervisor(config.restartStrategy, config.maxRestartsHandler)
supervisor.configure(config)
supervisor.start
supervisor
supervisor.start()
}
}
@ -147,7 +146,6 @@ sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (Act
case Supervise(actorRef, lifeCycle, registerAsRemoteService)
// actorRef.lifeCycle = lifeCycle THIS IS NOT COOL, BUT WAITING FOR https://www.assembla.com/spaces/akka/tickets/1124-supervisor-dsl-doesn-t-make-much-sense-after-the-introduction-of-props
supervisor.link(actorRef)
actorRef.start()
_childActors.add(actorRef) //TODO Why do we keep this here, mem leak?

View file

@ -45,7 +45,6 @@ import akka.japi.{ Creator, Procedure }
*
* public static void main(String[] args) {
* ActorRef actor = Actors.actorOf(SampleUntypedActor.class);
* actor.start();
* actor.tell("SendToSelf");
* actor.stop();
* }
@ -86,7 +85,7 @@ abstract class UntypedActor extends Actor {
/**
* User overridable callback.
* <p/>
* Is called when an Actor is started by invoking 'actor.start()'.
* Is called when an Actor is started, this only happens at most once in the life of an actor.
*/
override def preStart() {}

View file

@ -178,7 +178,7 @@ abstract class RemoteSupport extends ListenerManagement with RemoteServerModule
val eventHandler: ActorRef = {
implicit object format extends StatelessActorFormat[RemoteEventHandler]
val clazz = classOf[RemoteEventHandler]
val handler = Actor.actorOf(Props(clazz).withLocalOnly(true), clazz.getName)
val handler = new LocalActorRef(Props(clazz), clazz.getName, true)
// add the remote client and server listener that pipes the events to the event handler system
addListener(handler)
handler

View file

@ -49,7 +49,12 @@ object MessageDispatcher {
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class MessageDispatcher {
abstract class MessageDispatcher extends Serializable {
private def writeObject(out: java.io.ObjectOutputStream) {
(new Exception).printStackTrace()
throw new Exception("Damn you!")
}
import MessageDispatcher._

View file

@ -119,7 +119,7 @@ object EventHandler extends ListenerManagement {
defaultListeners foreach { listenerName
try {
ReflectiveAccess.getClassFor[Actor](listenerName) match {
case Right(actorClass) addListener(Actor.actorOf(Props(actorClass).withDispatcher(EventHandlerDispatcher)))
case Right(actorClass) addListener(new LocalActorRef(Props(actorClass).withDispatcher(EventHandlerDispatcher), newUuid.toString, systemService = true))
case Left(exception) throw exception
}
} catch {

View file

@ -117,7 +117,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
for (i 0 until requestedCapacity) yield {
val delegate = instance()
self link delegate
delegate.start()
delegate
}
}
case qty if qty < 0

View file

@ -219,17 +219,13 @@ object Routing {
val clusteringEnabled = ReflectiveAccess.ClusterModule.isEnabled
val localOnly = props.localOnly
if (!localOnly && !clusteringEnabled)
throw new IllegalArgumentException("Can't have clustered actor reference without the ClusterModule being enabled")
else if (clusteringEnabled && !props.localOnly)
ReflectiveAccess.ClusterModule.newClusteredActorRef(props).start()
if (clusteringEnabled && !props.localOnly)
ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
else {
if (props.connections.isEmpty)
throw new IllegalArgumentException("A routed actorRef can't have an empty connection set")
new RoutedActorRef(props).start()
new RoutedActorRef(props)
}
}
@ -270,7 +266,7 @@ object Routing {
RoutedProps.defaultFailureDetectorFactory,
actorAddress,
connections,
RoutedProps.defaultTimeout, true)).start()
RoutedProps.defaultTimeout, true))
}
}
@ -309,12 +305,6 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps) extends Abstrac
router.init(new RemoveConnectionOnFirstFailureLocalFailureDetector(routedProps.connections))
def start(): this.type = synchronized[this.type] {
if (_status == ActorRefInternals.UNSTARTED)
_status = ActorRefInternals.RUNNING
this
}
def stop() {
synchronized {
if (_status == ActorRefInternals.RUNNING) {
@ -323,6 +313,10 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps) extends Abstrac
}
}
}
/*If you start me up*/
if (_status == ActorRefInternals.UNSTARTED)
_status = ActorRefInternals.RUNNING
}
/**

View file

@ -249,7 +249,7 @@ object Duration {
* val d3 = d2 + 1.millisecond
* </pre>
*/
abstract class Duration {
abstract class Duration extends Serializable {
def length: Long
def unit: TimeUnit
def toNanos: Long

View file

@ -5,6 +5,7 @@
package akka.util
import akka.event.EventHandler
import java.io.{ PrintWriter, StringWriter }
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -39,6 +40,11 @@ object Helpers {
body
} catch {
case e: Throwable
val sw = new java.io.StringWriter()
var root = e
while (root.getCause ne null) root = e.getCause
root.printStackTrace(new java.io.PrintWriter(sw))
System.err.println(sw.toString)
EventHandler.error(e, this, e.toString)
throw e
}

View file

@ -26,7 +26,6 @@ trait ListenerManagement {
* The <code>listener</code> is started by this method if manageLifeCycleOfListeners yields true.
*/
def addListener(listener: ActorRef) {
if (manageLifeCycleOfListeners) listener.start()
listeners add listener
}

View file

@ -7,8 +7,7 @@ package akka.camel
import org.apache.camel.CamelContext
import akka.actor.Actor._
import akka.actor.Props
import akka.actor.ActorRef
import akka.actor._
import akka.camel.component.TypedActorComponent
/**
@ -33,8 +32,8 @@ private[camel] object TypedCamel {
* and re-uses the <code>activationTracker</code> of <code>service</code>.
*/
def onCamelServiceStart(service: CamelService) {
consumerPublisher = actorOf(Props(new TypedConsumerPublisher(service.activationTracker)).withLocalOnly(true))
publishRequestor = actorOf(Props(new TypedConsumerPublishRequestor).withLocalOnly(true))
consumerPublisher = new LocalActorRef(Props(new TypedConsumerPublisher(service.activationTracker)), newUuid.toString, true)
publishRequestor = new LocalActorRef(Props(new TypedConsumerPublishRequestor), newUuid.toString, true)
registerPublishRequestor

View file

@ -21,13 +21,13 @@ class TypedConsumerPublishRequestorTest extends JUnitSuite {
@Before
def setUp: Unit = {
publisher = actorOf(new TypedConsumerPublisherMock).start
requestor = actorOf(new TypedConsumerPublishRequestor).start
publisher = actorOf(new TypedConsumerPublisherMock)
requestor = actorOf(new TypedConsumerPublishRequestor)
requestor ! InitPublishRequestor(publisher)
consumer = actorOf(new Actor with Consumer {
def endpointUri = "mock:test"
protected def receive = null
}).start
})
}
@After

View file

@ -99,8 +99,8 @@ trait CamelContextLifecycle {
c context
t template
} {
c.start
t.start
c.start()
t.start()
_started = true
EventHandler.info(this, "Camel context started")
}

View file

@ -8,8 +8,7 @@ import java.util.concurrent.TimeUnit
import org.apache.camel.CamelContext
import akka.actor.Props
import akka.actor.Actor._
import akka.actor.{ newUuid, Props, LocalActorRef, Actor }
import akka.config.Config._
import akka.japi.{ SideEffect, Option JOption }
import akka.util.Bootable
@ -27,9 +26,9 @@ import TypedCamelAccess._
* @author Martin Krasser
*/
trait CamelService extends Bootable {
private[camel] val activationTracker = actorOf(Props(new ActivationTracker).withLocalOnly(true))
private[camel] val consumerPublisher = actorOf(Props(new ConsumerPublisher(activationTracker)).withLocalOnly(true))
private[camel] val publishRequestor = actorOf(Props(new ConsumerPublishRequestor).withLocalOnly(true))
private[camel] val activationTracker = new LocalActorRef(Props[ActivationTracker], newUuid.toString, true)
private[camel] val consumerPublisher = new LocalActorRef(Props(new ConsumerPublisher(activationTracker)), newUuid.toString, true)
private[camel] val publishRequestor = new LocalActorRef(Props(new ConsumerPublishRequestor), newUuid.toString, true)
private val serviceEnabled = config.getList("akka.enabled-modules").exists(_ == "camel")
@ -65,9 +64,6 @@ trait CamelService extends Bootable {
registerPublishRequestor
activationTracker.start
consumerPublisher.start
// send registration events for all (un)typed actors that have been registered in the past.
for (event PublishRequestor.pastActorRegisteredEvents) publishRequestor ! event
@ -179,10 +175,10 @@ trait CamelService extends Bootable {
(activationTracker ? SetExpectedDeactivationCount(count)).as[CountDownLatch].get
private[camel] def registerPublishRequestor: Unit =
registry.addListener(publishRequestor)
Actor.registry.addListener(publishRequestor)
private[camel] def unregisterPublishRequestor: Unit =
registry.removeListener(publishRequestor)
Actor.registry.removeListener(publishRequestor)
}
/**

View file

@ -173,7 +173,7 @@ trait ProducerSupport { this: Actor ⇒
*/
private def createSendProcessor = {
val sendProcessor = new SendProcessor(endpoint)
sendProcessor.start
sendProcessor.start()
sendProcessor
}
}

View file

@ -266,6 +266,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
val address = exchange.getExchangeId
def start = {
if (_status == ActorRefInternals.UNSTARTED)
_status = ActorRefInternals.RUNNING
this
}

View file

@ -32,7 +32,7 @@ public class ConsumerJavaTestBase {
public void shouldHandleExceptionThrownByActorAndGenerateCustomResponse() {
getMandatoryService().awaitEndpointActivation(1, new SideEffect() {
public void apply() {
actorOf(SampleErrorHandlingConsumer.class).start();
actorOf(SampleErrorHandlingConsumer.class);
}
});
String result = getMandatoryTemplate().requestBody("direct:error-handler-test-java", "hello", String.class);

View file

@ -18,13 +18,13 @@ class ConsumerPublishRequestorTest extends JUnitSuite {
@Before
def setUp: Unit = {
publisher = actorOf(new ConsumerPublisherMock).start
requestor = actorOf(new ConsumerPublishRequestor).start
publisher = actorOf(new ConsumerPublisherMock)
requestor = actorOf(new ConsumerPublishRequestor)
requestor ! InitPublishRequestor(publisher)
consumer = actorOf(new Actor with Consumer {
def endpointUri = "mock:test"
protected def receive = null
}).start.asInstanceOf[LocalActorRef]
}).asInstanceOf[LocalActorRef]
}
@After

View file

@ -21,17 +21,15 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
import CamelContextManager.mandatoryTemplate
import ConsumerScalaTest._
var service: CamelService = _
var service: CamelService = null
override protected def beforeAll = {
registry.local.shutdownAll
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")).start
actorOf(new TestConsumer("direct:publish-test-1"))
service.registerPublishRequestor
service.consumerPublisher.start
service.activationTracker.start
service.awaitEndpointActivation(1) {
service.start
} must be(true)
@ -43,7 +41,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
}
"A responding consumer" when {
val consumer = actorOf(new TestConsumer("direct:publish-test-2"))
var consumer: ActorRef = null
"started before starting the CamelService" must {
"support an in-out message exchange via its endpoint" in {
mandatoryTemplate.requestBody("direct:publish-test-1", "msg1") must equal("received msg1")
@ -57,7 +55,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.start
consumer = actorOf(new TestConsumer("direct:publish-test-2"))
} must be(true)
mandatoryTemplate.requestBody("direct:publish-test-2", "msg2") must equal("received msg2")
}
@ -78,11 +76,11 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
}
"A responding, untyped consumer" when {
val consumer = Actor.actorOf(classOf[SampleUntypedConsumer])
var consumer: ActorRef = null
"started" must {
"support an in-out message exchange via its endpoint" in {
service.awaitEndpointActivation(1) {
consumer.start
consumer = Actor.actorOf(classOf[SampleUntypedConsumer])
} must be(true)
mandatoryTemplate.requestBodyAndHeader("direct:test-untyped-consumer", "x", "test", "y") must equal("x y")
}
@ -103,7 +101,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
"receiving an in-out message exchange" must {
"lead to a TimeoutException" in {
service.awaitEndpointActivation(1) {
actorOf(Props(creator = () new TestBlocker("direct:publish-test-5"), timeout = Timeout(1000))).start
actorOf(Props(creator = () new TestBlocker("direct:publish-test-5"), timeout = Timeout(1000)))
} must be(true)
try {
@ -122,7 +120,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].start
actorOf[ErrorHandlingConsumer]
} must be(true)
mandatoryTemplate.requestBody("direct:error-handler-test", "hello") must equal("error: hello")
@ -131,7 +129,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].start
actorOf[RedeliveringConsumer]
} must be(true)
mandatoryTemplate.requestBody("direct:redelivery-test", "hello") must equal("accepted: hello")
@ -140,11 +138,13 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
}
"An non auto-acknowledging consumer" when {
val consumer = actorOf(new TestAckConsumer("direct:application-ack-test"))
"started" must {
"must support acknowledgements on application level" in {
var consumer: ActorRef = null
service.awaitEndpointActivation(1) {
consumer.start
consumer = actorOf(new TestAckConsumer("direct:application-ack-test"))
} must be(true)
val endpoint = mandatoryContext.getEndpoint("direct:application-ack-test", classOf[DirectEndpoint])
@ -163,13 +163,14 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
producer.process(exchange, handler)
latch.await(5, TimeUnit.SECONDS) must be(true)
consumer.stop
}
}
}
"A supervised consumer" must {
"be able to reply during receive" in {
val consumer = Actor.actorOf(new SupervisedConsumer("reply-channel-test-1")).start
val consumer = Actor.actorOf(new SupervisedConsumer("reply-channel-test-1"))
(consumer ? "succeed").get must equal("ok")
}
@ -181,7 +182,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
Supervise(consumer, Permanent) :: Nil))
val latch = new CountDownLatch(1)
val sender = Actor.actorOf(new Sender("pr", latch)).start
val sender = Actor.actorOf(new Sender("pr", latch))
consumer.!("fail")(Some(sender))
latch.await(5, TimeUnit.SECONDS) must be(true)
@ -195,7 +196,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)).start
val sender = Actor.actorOf(new Sender("ps", latch))
consumer.!("fail")(Some(sender))
latch.await(5, TimeUnit.SECONDS) must be(true)

View file

@ -32,7 +32,6 @@ 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))
producer.start
when("a test message is sent to the producer with ?")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
@ -46,7 +45,6 @@ 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"))
producer.start
when("a test message causing an exception is sent to the producer with ?")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
@ -62,7 +60,6 @@ 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)
producer.start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("TEST")
@ -75,7 +72,6 @@ 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"))
producer.start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("test")
@ -91,7 +87,6 @@ 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"))
producer.start
when("a test message is sent to the producer with ?")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
@ -104,7 +99,6 @@ 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"))
producer.start
when("a test message causing an exception is sent to the producer with ?")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
@ -122,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
val target = actorOf[ReplyingForwardTarget]
val producer = actorOf(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"))
@ -136,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
val target = actorOf[ReplyingForwardTarget]
val producer = actorOf(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"))
@ -152,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
val target = actorOf[ProducingForwardTarget]
val producer = actorOf(new TestForwarder("direct:producer-test-2", target))
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("received test")
@ -165,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
val target = actorOf[ProducingForwardTarget]
val producer = actorOf(new TestForwarder("direct:producer-test-2", target))
when("a test message causing an exception is sent to the producer with !")
mockEndpoint.expectedMessageCount(1)
@ -182,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
val target = actorOf[ReplyingForwardTarget]
val producer = actorOf(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"))
@ -196,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
val target = actorOf[ReplyingForwardTarget]
val producer = actorOf(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"))
@ -212,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
val target = actorOf[ProducingForwardTarget]
val producer = actorOf(new TestForwarder("direct:producer-test-3", target))
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("received test")
@ -225,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].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
val target = actorOf[ProducingForwardTarget]
val producer = actorOf(new TestForwarder("direct:producer-test-3", target))
when("a test message causing an exception is sent to the producer with !")
mockEndpoint.expectedMessageCount(1)
@ -279,7 +273,7 @@ object ProducerFeatureTest {
}
class TestRoute extends RouteBuilder {
val responder = actorOf[TestResponder].start
val responder = actorOf[TestResponder]
def configure {
from("direct:forward-test-1").to("mock:mock")
// for one-way messaging tests

View file

@ -31,7 +31,6 @@ class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with
scenario("produce message and receive normal response") {
given("a registered two-way producer")
val producer = actorOf(classOf[SampleUntypedReplyingProducer])
producer.start
when("a test message is sent to the producer with ?")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
@ -45,7 +44,6 @@ class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with
scenario("produce message and receive failure response") {
given("a registered two-way producer")
val producer = actorOf(classOf[SampleUntypedReplyingProducer])
producer.start
when("a test message causing an exception is sent to the producer with ?")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
@ -65,7 +63,6 @@ class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with
scenario("produce message and send normal response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val producer = actorOf(classOf[SampleUntypedForwardingProducer])
producer.start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("received test")

View file

@ -33,7 +33,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
import CamelContextManager.mandatoryTemplate
scenario("one-way communication") {
val actor = actorOf[Tester1].start
val actor = actorOf[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,12 +42,12 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
}
scenario("two-way communication") {
val actor = actorOf[Tester2].start
val actor = actorOf[Tester2]
assert(mandatoryTemplate.requestBody("actor:uuid:%s" format actor.uuid, "Martin") === "Hello Martin")
}
scenario("two-way communication with timeout") {
val actor = actorOf(Props[Tester3].withTimeout(Timeout(1))).start
val actor = actorOf(Props[Tester3].withTimeout(Timeout(1)))
intercept[RuntimeCamelException] {
mandatoryTemplate.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin")
}
@ -70,7 +70,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
import CamelContextManager.mandatoryTemplate
scenario("one-way communication") {
val actor = actorOf[Tester1].start
val actor = actorOf[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].start
val actor = actorOf[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").start
val actor = actorOf[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].start
val failWithException = actorOf[FailWithException].start
val failWithMessage = actorOf[FailWithMessage]
val failWithException = actorOf[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].start
val actor = actorOf[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].start
val actor = actorOf[Tester1]
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOnly)
@ -55,7 +55,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
def shouldSendMessageToActorAndReceiveResponseWithSyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
}).start
})
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")
@ -69,7 +69,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
def shouldSendMessageToActorAndReceiveResponseWithAsyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
}).start
})
val completion = expectAsyncCompletion
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
@ -85,7 +85,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
def shouldSendMessageToActorAndReceiveFailureWithAsyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = Failure(new Exception("testmsg"), Map("k3" -> "v3"))
}).start
})
val completion = expectAsyncCompletion
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
@ -102,7 +102,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
def shouldSendMessageToActorAndReceiveAckWithAsyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = akka.camel.Ack
}).start
})
val completion = expectAsyncCompletion
val endpoint = actorEndpoint("actor:uuid:%s?autoack=false" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOnly)
@ -117,8 +117,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
def shouldDynamicallyRouteMessageToActorWithDefaultId = {
val actor1 = actorOf[Tester1]("x")
val actor2 = actorOf[Tester1]("y")
actor1.start
actor2.start
actor1
actor2
val latch1 = (actor1 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
val latch2 = (actor2 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:id:%s" format actor1.address)
@ -141,8 +141,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
def shouldDynamicallyRouteMessageToActorWithoutDefaultId = {
val actor1 = actorOf[Tester1]("x")
val actor2 = actorOf[Tester1]("y")
actor1.start
actor2.start
actor1
actor2
val latch1 = (actor1 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
val latch2 = (actor2 ? SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:id:")
@ -164,8 +164,8 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldDynamicallyRouteMessageToActorWithDefaultUuid = {
val actor1 = actorOf[Tester1].start
val actor2 = actorOf[Tester1].start
val actor1 = actorOf[Tester1]
val actor2 = actorOf[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].start
val actor2 = actorOf[Tester1].start
val actor1 = actorOf[Tester1]
val actor2 = actorOf[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: Unit = {
val actor = actorOf[Tester1].start
val actor = actorOf[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: Unit = {
val actor = actorOf[Tester1].start
val actor = actorOf[Tester1]
val latch = (actor ? SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:")
intercept[ActorIdentifierNotSetException] {
@ -229,7 +229,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldSendMessageToActorAndTimeout(): Unit = {
val actor = actorOf(Props[Tester3].withTimeout(Timeout(1))).start
val actor = actorOf(Props[Tester3].withTimeout(Timeout(1)))
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")

View file

@ -287,7 +287,7 @@ class DefaultClusterNode private[akka] (
}
}), "akka.cluster.RemoteClientLifeCycleListener")
private[cluster] lazy val remoteDaemon = actorOf(Props(new RemoteClusterDaemon(this)).copy(dispatcher = new PinnedDispatcher(), localOnly = true), RemoteClusterDaemon.Address)
private[cluster] lazy val remoteDaemon = new LocalActorRef(Props(new RemoteClusterDaemon(this)).copy(dispatcher = new PinnedDispatcher()), RemoteClusterDaemon.Address, systemService = true)
private[cluster] lazy val remoteDaemonSupervisor = Supervisor(
SupervisorConfig(
@ -295,7 +295,7 @@ class DefaultClusterNode private[akka] (
Supervise(
remoteDaemon,
Permanent)
:: Nil))
:: Nil)).start()
lazy val remoteService: RemoteSupport = {
val remote = new akka.cluster.netty.NettyRemoteSupport
@ -502,7 +502,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, 0, Transient, false, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), 0, Transient, false, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -510,7 +510,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, 0, replicationScheme, false, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), 0, replicationScheme, false, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -518,7 +518,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, replicationFactor, Transient, false, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, Transient, false, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -526,7 +526,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, replicationFactor, replicationScheme, false, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, replicationScheme, false, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -534,7 +534,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, 0, Transient, serializeMailbox, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), 0, Transient, serializeMailbox, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -542,7 +542,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, 0, replicationScheme, serializeMailbox, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), 0, replicationScheme, serializeMailbox, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -550,7 +550,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, replicationFactor, Transient, serializeMailbox, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, Transient, serializeMailbox, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -558,7 +558,7 @@ class DefaultClusterNode private[akka] (
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress).start, replicationFactor, replicationScheme, serializeMailbox, serializer)
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, replicationScheme, serializeMailbox, serializer)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -814,7 +814,6 @@ class DefaultClusterNode private[akka] (
// create ADDRESS -> NODE mapping
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToNodesPathFor(actorAddress, nodeName)))
actorRef.start()
actorRef
}
}
@ -1318,7 +1317,7 @@ class DefaultClusterNode private[akka] (
EventHandler.debug(this, "Setting up connection to node with nodename [%s] and address [%s]".format(node, address))
val clusterDaemon = remoteService.actorFor(
RemoteClusterDaemon.Address, address.getHostName, address.getPort).start()
RemoteClusterDaemon.Address, address.getHostName, address.getPort)
newConnections = newConnections + (node -> (address, clusterDaemon))
change = true
}
@ -1784,7 +1783,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
if (Cluster.shouldCompressData) LZF.uncompress(bytes)
else bytes
val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start()
val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid)
cluster.remoteService.register(actorAddress, snapshotActorRef)
// FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently
@ -1864,25 +1863,35 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}
def handle_fun0_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
actorOf(Props(
self { case f: Function0[_] try { f() } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) ! payloadFor(message, classOf[Function0[Unit]])
new LocalActorRef(
Props(
self {
case f: Function0[_] try { f() } finally { self.stop() }
}).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
}
def handle_fun0_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
actorOf(Props(
self { case f: Function0[_] try { self.reply(f()) } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) forward payloadFor(message, classOf[Function0[Any]])
new LocalActorRef(
Props(
self {
case f: Function0[_] try { self.reply(f()) } finally { self.stop() }
}).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
}
def handle_fun1_arg_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
actorOf(Props(
self { case (fun: Function[_, _], param: Any) try { fun.asInstanceOf[Any Unit].apply(param) } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
new LocalActorRef(
Props(
self {
case (fun: Function[_, _], param: Any) try { fun.asInstanceOf[Any Unit].apply(param) } finally { self.stop() }
}).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
}
def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
actorOf(Props(
new LocalActorRef(
Props(
self {
case (fun: Function[_, _], param: Any) try { self.reply(fun.asInstanceOf[Any Any](param)) } finally { self.stop() }
}).copy(dispatcher = computeGridDispatcher, localOnly = true)) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
}).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
}
def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) {

View file

@ -60,7 +60,7 @@ object ClusterActorRef {
.withDeployId(actorAddress)
.withTimeout(timeout)
.withRouter(routerFactory)
.withFailureDetector(failureDetectorFactory)).start()
.withFailureDetector(failureDetectorFactory))
}
/**
@ -112,14 +112,6 @@ private[akka] class ClusterActorRef(props: RoutedProps) extends AbstractRoutedAc
connections.failOver(from, to)
}
def start(): this.type = synchronized[this.type] {
if (_status == ActorRefInternals.UNSTARTED) {
_status = ActorRefInternals.RUNNING
Actor.registry.local.registerClusterActorRef(this)
}
this
}
def stop() {
synchronized {
if (_status == ActorRefInternals.RUNNING) {
@ -132,4 +124,10 @@ private[akka] class ClusterActorRef(props: RoutedProps) extends AbstractRoutedAc
}
}
}
/* If you start me up */
if (_status == ActorRefInternals.UNSTARTED) {
_status = ActorRefInternals.RUNNING
Actor.registry.local.registerClusterActorRef(this)
}
}

View file

@ -168,7 +168,7 @@ object ClusterDeployer extends ActorDeployer {
case Deploy(_, _, _, _, Local) | Deploy(_, _, _, _, _: Local) //TODO LocalDeployer.deploy(deployment)??
case Deploy(address, recipe, routing, _, _) // cluster deployment
/*TODO recipe foreach { r ⇒
Deployer.newClusterActorRef(() Actor.actorOf(r.implementationClass), address, deployment).start()
Deployer.newClusterActorRef(() Actor.actorOf(r.implementationClass), address, deployment)
}*/
val path = deploymentAddressPath.format(address)
try {

View file

@ -88,9 +88,8 @@ object LocalCluster {
*/
def shutdownLocalCluster() {
withPrintStackTraceOnError {
EventHandler.info(this, "Shuts down local cluster")
zkServer.get.foreach(_.shutdown())
zkServer.set(None)
EventHandler.debug(this, "Shuts down local cluster")
zkServer.getAndSet(None).foreach(_.shutdown())
}
}

View file

@ -4,8 +4,8 @@
package akka.cluster
import akka.actor.{ Actor, ActorRef, Props }
import Actor._
import akka.actor.{ newUuid, Actor, ActorRef, Props, LocalActorRef }
import akka.actor.Actor._
import akka.cluster._
import akka.routing._
import akka.event.EventHandler
@ -24,7 +24,7 @@ object RemoteFailureDetector {
private case class Register(strategy: RemoteFailureListener, address: InetSocketAddress) extends FailureDetectorEvent
private case class Unregister(strategy: RemoteFailureListener, address: InetSocketAddress) extends FailureDetectorEvent
private[akka] val registry = actorOf(Props(new Registry).copy(dispatcher = new PinnedDispatcher(), localOnly = true))
private[akka] val registry = new LocalActorRef(Props[Registry].copy(dispatcher = new PinnedDispatcher()), newUuid.toString, systemService = true)
def register(strategy: RemoteFailureListener, address: InetSocketAddress) = registry ! Register(strategy, address)

View file

@ -769,10 +769,8 @@ trait NettyRemoteServerModule extends RemoteServerModule {
}
private def register[Key](id: Key, actorRef: ActorRef, registry: ConcurrentHashMap[Key, ActorRef]) {
if (_isRunning.isOn) {
if (_isRunning.isOn)
registry.put(id, actorRef) //TODO change to putIfAbsent
if (!actorRef.isRunning) actorRef.start()
}
}
/**
@ -1078,7 +1076,7 @@ class RemoteServerHandler(
"Looking up a remotely available actor for address [%s] on node [%s]"
.format(address, Config.nodename))
val actorRef = Actor.createActor(address, () createSessionActor(actorInfo, channel))
val actorRef = Actor.createActor(address, () createSessionActor(actorInfo, channel), false)
if (actorRef eq null) throw new IllegalActorStateException("Could not find a remote actor with address [" + address + "] or uuid [" + uuid + "]")
@ -1098,9 +1096,8 @@ class RemoteServerHandler(
case null null
case factory
val actorRef = factory()
actorRef.uuid = parseUuid(uuid) //FIXME is this sensible?
sessionActors.get(channel).put(address, actorRef)
actorRef.start() //Start it where's it's created
actorRef //Start it where's it's created
}
case sessionActor sessionActor
}

View file

@ -230,7 +230,7 @@ object ActorSerialization {
//
)
val receiveTimeout = if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None //TODO FIXME, I'm expensive and slow
val receiveTimeout = if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None
val ar = new LocalActorRef(actorUuid, protocol.getAddress, props, receiveTimeout, storedHotswap)

View file

@ -42,7 +42,7 @@
// }
// barrier("create-actor-on-node1", NrOfNodes) {
// val actorRef = Actor.actorOf[HelloWorld]("hello-world-write-behind-nosnapshot").start()
// val actorRef = Actor.actorOf[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").start()
// val actorRef = Actor.actorOf[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").start()
// val actorRef = Actor.actorOf[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").start()
// val actorRef = Actor.actorOf[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

@ -45,11 +45,11 @@ class HomeNodeMultiJvmNode2 extends ClusterTestNode {
Cluster.node.start()
barrier("waiting-for-begin", NrOfNodes).await()
val actorNode1 = Actor.actorOf[SomeActor]("service-node1").start()
val actorNode1 = Actor.actorOf[SomeActor]("service-node1")
val name1 = (actorNode1 ? "identify").get.asInstanceOf[String]
name1 must equal("node1")
val actorNode2 = Actor.actorOf[SomeActor]("service-node2").start()
val actorNode2 = Actor.actorOf[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").start().asInstanceOf[ClusterActorRef]
val actor = Actor.actorOf[SomeActor]("service-hello").asInstanceOf[ClusterActorRef]
actor.isRunning must be(true)
val result = (actor ? "identify").get

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").start()
val actorNode1 = Actor.actorOf[SomeActor]("service-node1")
val name1 = (actorNode1 ? "identify").get.asInstanceOf[String]
name1 must equal("node1")
val actorNode2 = Actor.actorOf[SomeActor]("service-node2").start()
val actorNode2 = Actor.actorOf[SomeActor]("service-node2")
val name2 = (actorNode2 ? "identify").get.asInstanceOf[String]
name2 must equal("node2")

View file

@ -26,7 +26,7 @@ object ScatterGatherFailoverMultiJvmSpec {
Thread.sleep(2000)
Cluster.node.shutdown()
}
}.start()
}
def receive = {
case Shutdown(None) shutdownNode

View file

@ -21,12 +21,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
describe("Serializable actor") {
it("should be able to serialize and de-serialize a stateful actor with a given serializer") {
val actor1 = actorOf(Props[MyJavaSerializableActor].withLocalOnly(true)).asInstanceOf[LocalActorRef]
val actor1 = new LocalActorRef(Props[MyJavaSerializableActor], newUuid.toString, true)
(actor1 ? "hello").get should equal("world 1")
(actor1 ? "hello").get should equal("world 2")
val bytes = toBinary(actor1)
val actor2 = fromBinary(bytes).start().asInstanceOf[LocalActorRef]
val actor2 = fromBinary(bytes).asInstanceOf[LocalActorRef]
(actor2 ? "hello").get should equal("world 3")
actor2.receiveTimeout should equal(Some(1000))
@ -36,7 +36,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
it("should be able to serialize and deserialize a MyStatelessActorWithMessagesInMailbox") {
val actor1 = actorOf(Props[MyStatelessActorWithMessagesInMailbox].withLocalOnly(true)).asInstanceOf[LocalActorRef]
val actor1 = new LocalActorRef(Props[MyStatelessActorWithMessagesInMailbox], newUuid.toString, true)
for (i 1 to 10) actor1 ! "hello"
actor1.getDispatcher.mailboxSize(actor1) should be > (0)
@ -54,17 +54,8 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
it("should be able to serialize and deserialize a PersonActorWithMessagesInMailbox") {
val p1 = Person("debasish ghosh", 25, SerializeSpec.Address("120", "Monroe Street", "Santa Clara", "95050"))
val actor1 = actorOf(Props[PersonActorWithMessagesInMailbox].withLocalOnly(true)).asInstanceOf[LocalActorRef]
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
val actor1 = new LocalActorRef(Props[PersonActorWithMessagesInMailbox], newUuid.toString, true)
for (i 1 to 10) actor1 ! p1
actor1.getDispatcher.mailboxSize(actor1) should be > (0)
val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
@ -98,7 +89,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
describe("serialize actor that accepts protobuf message") {
it("should serialize") {
val actor1 = actorOf(Props[MyActorWithProtobufMessagesInMailbox].withLocalOnly(true)).asInstanceOf[LocalActorRef]
val actor1 = new LocalActorRef(Props[MyActorWithProtobufMessagesInMailbox], newUuid.toString, true)
val msg = MyMessage(123, "debasish ghosh", true)
val b = ProtobufProtocol.MyMessage.newBuilder.setId(msg.id).setName(msg.name).setStatus(msg.status).build
for (i 1 to 10) actor1 ! b

View file

@ -62,7 +62,7 @@ object Pi extends App {
//#create-workers
// create the workers
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker].start())
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.actorOf(
@ -72,7 +72,7 @@ object Pi extends App {
.withDeployId("pi")
)
loadBalancerActor(CyclicIterator(workers)).start()
loadBalancerActor(CyclicIterator(workers))
//#create-workers
//#master-receive
@ -123,7 +123,7 @@ object Pi extends App {
// create the master
val master = actorOf(
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)).start()
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
// start the calculation
master ! Calculate

View file

@ -338,7 +338,7 @@ The master actor is a little bit more involved. In its constructor we need to cr
// create the workers
final ActorRef[] workers = new ActorRef[nrOfWorkers];
for (int i = 0; i < nrOfWorkers; i++) {
workers[i] = actorOf(Worker.class).start();
workers[i] = actorOf(Worker.class);
}
// wrap them with a load-balancing router
@ -346,7 +346,7 @@ The master actor is a little bit more involved. In its constructor we need to cr
public UntypedActor create() {
return new PiRouter(workers);
}
}).start();
});
}
}
@ -360,8 +360,7 @@ One thing to note is that we used two different versions of the ``actorOf`` meth
The actor's life-cycle is:
- Created -- ``Actor.actorOf[MyActor]`` -- can **not** receive messages
- Started -- ``actorRef.start()`` -- can receive messages
- Created & Started -- ``Actor.actorOf[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.
@ -406,7 +405,7 @@ Here is the master actor::
// create the workers
final ActorRef[] workers = new ActorRef[nrOfWorkers];
for (int i = 0; i < nrOfWorkers; i++) {
workers[i] = actorOf(Worker.class).start();
workers[i] = actorOf(Worker.class);
}
// wrap them with a load-balancing router
@ -414,7 +413,7 @@ Here is the master actor::
public UntypedActor create() {
return new PiRouter(workers);
}
}).start();
});
}
// message handler
@ -501,7 +500,7 @@ Now the only thing that is left to implement is the runner that should bootstrap
public UntypedActor create() {
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
}
}).start();
});
// start the calculation
master.tell(new Calculate());
@ -631,7 +630,7 @@ Before we package it up and run it, let's take a look at the full code now, with
// create the workers
final ActorRef[] workers = new ActorRef[nrOfWorkers];
for (int i = 0; i < nrOfWorkers; i++) {
workers[i] = actorOf(Worker.class).start();
workers[i] = actorOf(Worker.class);
}
// wrap them with a load-balancing router
@ -639,7 +638,7 @@ Before we package it up and run it, let's take a look at the full code now, with
public UntypedActor create() {
return new PiRouter(workers);
}
}).start();
});
}
// message handler
@ -697,7 +696,7 @@ Before we package it up and run it, let's take a look at the full code now, with
public UntypedActor create() {
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
}
}).start();
});
// start the calculation
master.tell(new Calculate());

View file

@ -254,10 +254,10 @@ 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].start())
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start()
val router = Routing.loadBalancerActor(CyclicIterator(workers))
As you can see we are using the ``actorOf`` factory method to create actors, this method returns as an ``ActorRef`` which is a reference to our newly created actor. This method is available in the ``Actor`` object but is usually imported::
@ -268,7 +268,7 @@ There are two versions of ``actorOf``; one of them taking a actor type and the o
The actor's life-cycle is:
- Created -- ``Actor.actorOf[MyActor]`` -- can **not** receive messages
- Started -- ``actorRef.start()`` -- can receive messages
- Started -- ``actorRef`` -- 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.
@ -290,10 +290,10 @@ Here is the master actor::
var start: Long = _
// create the workers
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker].start())
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start()
val router = Routing.loadBalancerActor(CyclicIterator(workers))
def receive = { ... }
@ -366,7 +366,7 @@ The ``Pi`` object is a perfect container module for our actors and messages, so
// create the master
val master = actorOf(
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)).start()
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
// start the calculation
master ! Calculate

View file

@ -250,10 +250,10 @@ Creating the master
The master actor is a little bit more involved. In its constructor we need to create the workers (the ``Worker`` actors) and start them. We will also wrap them in a load-balancing router to make it easier to spread out the work evenly between the workers. Let's do that first::
// create the workers
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker].start())
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start()
val router = Routing.loadBalancerActor(CyclicIterator(workers))
As you can see we are using the ``actorOf`` factory method to create actors, this method returns as an ``ActorRef`` which is a reference to our newly created actor. This method is available in the ``Actor`` object but is usually imported::
@ -263,8 +263,7 @@ There are two versions of ``actorOf``; one of them taking a actor type and the o
The actor's life-cycle is:
- Created -- ``Actor.actorOf[MyActor]`` -- can **not** receive messages
- Started -- ``actorRef.start()`` -- can receive messages
- Created & Started -- ``Actor.actorOf[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.
@ -286,10 +285,10 @@ Here is the master actor::
var start: Long = _
// create the workers
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker].start())
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start()
val router = Routing.loadBalancerActor(CyclicIterator(workers))
def receive = { ... }
@ -362,7 +361,7 @@ The ``Pi`` object is a perfect container module for our actors and messages, so
// create the master
val master = actorOf(
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)).start()
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
// start the calculation
master ! Calculate
@ -428,10 +427,10 @@ But before we package it up and run it, let's take a look at the full code now,
var start: Long = _
// create the workers
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker].start())
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start()
val router = Routing.loadBalancerActor(CyclicIterator(workers))
// message handler
def receive = {
@ -476,7 +475,7 @@ But before we package it up and run it, let's take a look at the full code now,
// create the master
val master = actorOf(
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)).start()
new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
// start the calculation
master ! Calculate

View file

@ -77,5 +77,5 @@ The above actor can be added as listener of registry events:
import static akka.actor.Actors.*;
ActorRef listener = actorOf(RegistryListener.class).start();
ActorRef listener = actorOf(RegistryListener.class);
registry().addListener(listener);

View file

@ -174,7 +174,6 @@ Creating a PriorityDispatcher using PriorityGenerator:
// We create a new Priority dispatcher and seed it with the priority generator
ref.setDispatcher(new PriorityDispatcher("foo", gen));
ref.start(); // Start the actor
ref.getDispatcher().suspend(ref); // Suspending the actor so it doesn't start to treat the messages before we have enqueued all of them :-)
ref.tell("lowpriority");
ref.tell("lowpriority");

View file

@ -233,7 +233,7 @@ Here is the API and how to use it from within an 'Actor':
getContext().unlink(actorRef);
// starts and links Actors atomically
getContext().link(actorRef).start();
getContext().link(actorRef);
// spawns (creates and starts) actors
getContext().spawn(MyActor.class);

View file

@ -38,7 +38,7 @@ Here is how to start up the server and specify the hostname and port in the a
import static akka.actor.Actors.*;
remote().start();
remote();
// Specify the classloader to use to load the remote class (actor)
remote().start(classLoader);

View file

@ -27,8 +27,8 @@ An UntypedDispatcher is an actor that routes incoming messages to outbound actor
}
public class MyRouter extends UntypedRouter {
private ActorRef pinger = actorOf(Pinger.class).start();
private ActorRef ponger = actorOf(Ponger.class).start();
private ActorRef pinger = actorOf(Pinger.class);
private ActorRef ponger = actorOf(Ponger.class);
//Route Ping-messages to the pinger, and Pong-messages to the ponger
public ActorRef route(Object message) {
@ -38,7 +38,7 @@ An UntypedDispatcher is an actor that routes incoming messages to outbound actor
}
}
ActorRef router = actorOf(MyRouter.class).start();
ActorRef router = actorOf(MyRouter.class);
router.tell("Ping"); //Prints "Pinger: Ping"
router.tell("Pong"); //Prints "Ponger: Pong"
@ -71,8 +71,8 @@ An UntypedLoadBalancer is an actor that forwards messages it receives to a bound
//Our load balancer, sends messages to a pinger, then a ponger, rinse and repeat.
public class MyLoadBalancer extends UntypedLoadBalancer {
private InfiniteIterator<ActorRef> actors = new CyclicIterator<ActorRef>(asList(
actorOf(Pinger.class).start(),
actorOf(Ponger.class).start()
actorOf(Pinger.class),
actorOf(Ponger.class)
));
public InfiniteIterator<ActorRef> seq() {
@ -80,7 +80,7 @@ An UntypedLoadBalancer is an actor that forwards messages it receives to a bound
}
}
ActorRef balancer = actorOf(MyLoadBalancer.class).start();
ActorRef balancer = actorOf(MyLoadBalancer.class);
balancer.tell("Pong"); //Prints "Pinger: Pong"
balancer.tell("Ping"); //Prints "Ponger: Ping"
balancer.tell("Ping"); //Prints "Pinger: Ping"

View file

@ -61,7 +61,6 @@ The following JUnit snippet first creates an actor using the default constructor
@Test public void mustBeAbleToSerializeAfterCreateActorRefFromClass() {
ActorRef ref = Actors.actorOf(SerializationTestActor.class);
assertNotNull(ref);
ref.start();
try {
Object result = ref.sendRequestReply("Hello");
assertEquals("got it!", result);
@ -73,7 +72,7 @@ The following JUnit snippet first creates an actor using the default constructor
byte[] bytes = toBinaryJ(ref, f, false);
ActorRef r = fromBinaryJ(bytes, f);
assertNotNull(r);
r.start();
try {
Object result = r.sendRequestReply("Hello");
assertEquals("got it!", result);
@ -151,7 +150,6 @@ Step 3: Serialize and de-serialize
@Test public void mustBeAbleToSerializeAStatefulActor() {
ActorRef ref = Actors.actorOf(MyUntypedActor.class);
assertNotNull(ref);
ref.start();
try {
Object result = ref.sendRequestReply("hello");
assertEquals("world 1", result);
@ -165,7 +163,6 @@ Step 3: Serialize and de-serialize
byte[] bytes = toBinaryJ(ref, f, false);
ActorRef r = fromBinaryJ(bytes, f);
assertNotNull(r);
r.start();
try {
Object result = r.sendRequestReply("hello");
assertEquals("world 3", result);

View file

@ -303,7 +303,7 @@ Here is an example of using ``retry`` to block until an account has enough money
final Ref<Double> account1 = new Ref<Double>(100.0);
final Ref<Double> account2 = new Ref<Double>(100.0);
ActorRef transferer = Actors.actorOf(Transferer.class).start();
ActorRef transferer = Actors.actorOf(Transferer.class);
transferer.tell(new Transfer(account1, account2, 500.0));
// Transferer: not enough money - retrying
@ -426,7 +426,7 @@ You can also have two alternative blocking transactions, one of which can succee
final Ref<Integer> left = new Ref<Integer>(100);
final Ref<Integer> right = new Ref<Integer>(100);
ActorRef brancher = Actors.actorOf(Brancher.class).start();
ActorRef brancher = Actors.actorOf(Brancher.class);
brancher.tell(new Branch(left, right, 500));
// not enough on left - retrying

View file

@ -102,8 +102,8 @@ Here is an example of coordinating two simple counter UntypedActors so that they
.. code-block:: java
ActorRef counter1 = actorOf(Counter.class).start();
ActorRef counter2 = actorOf(Counter.class).start();
ActorRef counter1 = actorOf(Counter.class);
ActorRef counter2 = actorOf(Counter.class);
counter1.tell(new Coordinated(new Increment(counter2)));

View file

@ -42,7 +42,6 @@ Creating an Actor is done using the 'akka.actor.Actors.actorOf' factory method.
.. code-block:: java
ActorRef myActor = Actors.actorOf(SampleUntypedActor.class);
myActor.start();
Normally you would want to import the 'actorOf' method like this:
@ -57,7 +56,7 @@ You can also create & start the actor in one statement:
.. code-block:: java
ActorRef myActor = actorOf(SampleUntypedActor.class).start();
ActorRef myActor = actorOf(SampleUntypedActor.class);
The call to 'actorOf' returns an instance of 'ActorRef'. This is a handle to the 'UntypedActor' instance which you can use to interact with the Actor, like send messages to it etc. more on this shortly. The 'ActorRef' is immutable and has a one to one relationship with the Actor it represents. The 'ActorRef' is also serializable and network-aware. This means that you can serialize it, send it over the wire and use it on a remote host and it will still be representing the same Actor on the original node, across the network.
@ -315,20 +314,13 @@ Promise represents the write-side of a Future, enabled by the methods
Starting actors
---------------
Actors are started by invoking the start method.
Actors are started when they are created by invoking the actorOf method.
.. code-block:: java
ActorRef actor = actorOf(SampleUntypedActor.class);
myActor.start();
You can create and start the Actor in a one liner like this:
.. code-block:: java
ActorRef actor = actorOf(SampleUntypedActor.class).start();
When you start the actor then it will automatically call the 'preStart' callback method on the 'UntypedActor'. This is an excellent place to add initialization code for the actor.
When you create the actor then it will automatically call the 'preStart' callback method on the 'UntypedActor'. This is an excellent place to add initialization code for the actor.
.. code-block:: java

View file

@ -62,7 +62,7 @@ one-liner. Here's an example.
}
// start and expose actor via tcp
val myActor = actorOf[MyActor].start
val myActor = actorOf[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
@ -354,7 +354,7 @@ Actors (untyped)
Publishing a consumer actor at its Camel endpoint occurs when the actor is
started. Publication is done asynchronously; setting up an endpoint (more
precisely, the route from that endpoint to the actor) may still be in progress
after the ActorRef.start method returned.
after the ActorRef method returned.
**Scala**
@ -362,8 +362,7 @@ after the ActorRef.start method returned.
import akka.actor.Actor._
val actor = actorOf[Consumer1] // create Consumer actor
actor.start // activate endpoint in background
val actor = actorOf[Consumer1] // create Consumer actor and activate endpoint in background
**Java**
@ -372,8 +371,7 @@ after the ActorRef.start method returned.
import static akka.actor.Actors.*;
import akka.actor.ActorRef;
ActorRef actor = actorOf(Consumer1.class); // create Consumer actor
actor.start(); // activate endpoint in background
ActorRef actor = actorOf(Consumer1.class); // create Consumer actor and activate endpoint in background
Typed actors
@ -547,7 +545,7 @@ still in progress after the ``ActorRef.stop`` method returned.
import akka.actor.Actor._
val actor = actorOf[Consumer1] // create Consumer actor
actor.start // activate endpoint in background
actor // activate endpoint in background
// ...
actor.stop // deactivate endpoint in background
@ -558,8 +556,7 @@ still in progress after the ``ActorRef.stop`` method returned.
import static akka.actor.Actors.*;
import akka.actor.ActorRef;
ActorRef actor = actorOf(Consumer1.class); // create Consumer actor
actor.start(); // activate endpoint in background
ActorRef actor = actorOf(Consumer1.class); // create Consumer actor and activate endpoint in background
// ...
actor.stop(); // deactivate endpoint in background
@ -958,7 +955,7 @@ actors (see below) is an initialized and started CamelContextManager.
import akka.camel.CamelContextManager;
CamelContextManager.init(); // optionally takes a CamelContext as argument
CamelContextManager.start(); // starts the managed CamelContext
CamelContextManager; // starts the managed CamelContext
For using producer actors, application may also start a CamelService. This will
not only setup a CamelContextManager behind the scenes but also register
@ -1031,7 +1028,7 @@ used.
import akka.actor.Actor._
import akka.actor.ActorRef
val producer = actorOf[Producer1].start
val producer = actorOf[Producer1]
val response = producer !! "akka rocks"
val body = response.bodyAs[String]
@ -1043,7 +1040,7 @@ used.
import static akka.actor.Actors.*;
import akka.camel.Message;
ActorRef producer = actorOf(Producer1.class).start();
ActorRef producer = actorOf(Producer1.class);
Message response = (Message)producer.sendRequestReply("akka rocks");
String body = response.getBodyAs(String.class)
@ -1160,7 +1157,7 @@ argument.
ActorRef target = ...
ActorRef producer = actorOf(new Producer1Factory(target));
producer.start();
producer;
Before producing messages to endpoints, producer actors can pre-process them by
overriding the receiveBeforeProduce method (Scala) or onReceiveBeforeProduce
@ -1949,7 +1946,7 @@ ends at the target actor.
import akka.camel.{Message, CamelContextManager, CamelServiceManager}
object CustomRouteExample extends Application {
val target = actorOf[CustomRouteTarget].start
val target = actorOf[CustomRouteTarget]
CamelServiceManager.startCamelService
CamelContextManager.mandatoryContext.addRoutes(new CustomRouteBuilder(target.uuid))
@ -1985,7 +1982,7 @@ ends at the target actor.
public class CustomRouteExample {
public static void main(String... args) throws Exception {
ActorRef target = actorOf(CustomRouteTarget.class).start();
ActorRef target = actorOf(CustomRouteTarget.class);
CamelServiceManager.startCamelService();
CamelContextManager.getMandatoryContext().addRoutes(new CustomRouteBuilder(target.getUuid()));
}
@ -2550,9 +2547,9 @@ as shown in the following snippet (see also `sample.camel.Boot`_).
}
// Wire and start the example actors
val httpTransformer = actorOf(new HttpTransformer).start
val httpProducer = actorOf(new HttpProducer(httpTransformer)).start
val httpConsumer = actorOf(new HttpConsumer(httpProducer)).start
val httpTransformer = actorOf(new HttpTransformer)
val httpProducer = actorOf(new HttpProducer(httpTransformer))
val httpConsumer = actorOf(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
@ -2645,10 +2642,6 @@ follows.
val producer = actorOf[Producer1]
val mediator = actorOf(new Transformer(producer))
val consumer = actorOf(new Consumer3(mediator))
producer.start
mediator.start
consumer.start
}
class CustomRouteBuilder extends RouteBuilder {
@ -2750,11 +2743,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)).start
val jmsSubscriber2 = actorOf(new Subscriber("jms-subscriber-2", jmsUri)).start
val jmsPublisher = actorOf(new Publisher("jms-publisher", jmsUri)).start
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 jmsPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/jms", jmsPublisher)).start
val jmsPublisherBridge = actorOf(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
@ -2847,10 +2840,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)).start
val cometdPublisher = actorOf(new Publisher("cometd-publisher", cometdUri)).start
val cometdSubscriber = actorOf(new Subscriber("cometd-subscriber", cometdUri))
val cometdPublisher = actorOf(new Publisher("cometd-publisher", cometdUri))
val cometdPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/cometd", cometdPublisher)).start
val cometdPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/cometd", cometdPublisher))
}
@ -2892,12 +2885,9 @@ seconds:
// start the Camel service
startCamelService
// create a quartz actor
// create and start a quartz actor
val myActor = actorOf[MyQuartzActor]
// start the quartz actor
myActor.start
} // end main
} // end MyQuartzActor

View file

@ -42,7 +42,7 @@ You can also start it in the same statement:
.. code-block:: scala
val a = actorOf[MyActor].start
val a = actorOf[MyActor]
Creating Actors with non-default constructor
--------------------------------------------

View file

@ -103,5 +103,5 @@ The above actor can be added as listener of registry events:
import akka.actor._
import akka.actor.Actor._
val listener = actorOf[RegistryListener].start()
val listener = actorOf[RegistryListener]
registry.addListener(listener)

View file

@ -50,7 +50,6 @@ Creating Actors
.. code-block:: scala
val myActor = Actor.actorOf[MyActor]
myActor.start()
Normally you would want to import the ``actorOf`` method like this:
@ -62,12 +61,6 @@ Normally you would want to import the ``actorOf`` method like this:
To avoid prefixing it with ``Actor`` every time you use it.
You can also start it in the same statement:
.. code-block:: scala
val myActor = actorOf[MyActor].start()
The call to ``actorOf`` returns an instance of ``ActorRef``. This is a handle to the ``Actor`` instance which you can use to interact with the ``Actor``. The ``ActorRef`` is immutable and has a one to one relationship with the Actor it represents. The ``ActorRef`` is also serializable and network-aware. This means that you can serialize it, send it over the wire and use it on a remote host and it will still be representing the same Actor on the original node, across the network.
Creating Actors with non-default constructor
@ -79,7 +72,7 @@ Here is an example:
.. code-block:: scala
val a = actorOf(new MyActor(..)).start() // allows passing in arguments into the MyActor constructor
val a = actorOf(new MyActor(..)) // allows passing in arguments into the MyActor constructor
Running a block of code asynchronously
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
@ -440,20 +433,14 @@ This mechanism also work for hotswapped receive functions. Every time a ``HotSwa
Starting actors
---------------
Actors are started by invoking the ``start`` method.
Actors are created & started by invoking the ``actorOf`` method.
.. code-block:: scala
val actor = actorOf[MyActor]
actor.start()
actor
You can create and start the ``Actor`` in a one liner like this:
.. code-block:: scala
val actor = actorOf[MyActor].start()
When you start the ``Actor`` then it will automatically call the ``def preStart`` callback method on the ``Actor`` trait. This is an excellent place to add initialization code for the actor.
When you create the ``Actor`` then it will automatically call the ``def preStart`` callback method on the ``Actor`` trait. This is an excellent place to add initialization code for the actor.
.. code-block:: scala
@ -555,7 +542,7 @@ Here is another little cute example of ``become`` and ``unbecome`` in action:
}
}
val swap = actorOf[Swapper].start()
val swap = actorOf[Swapper]
swap ! Swap // prints Hi
swap ! Swap // prints Ho

View file

@ -154,15 +154,11 @@ Creating a PriorityDispatcher using PriorityGenerator:
}
val a = Actor.actorOf( // We create a new Actor that just prints out what it processes
new Actor {
Props(new Actor {
def receive = {
case x => println(x)
}
})
// We create a new Priority dispatcher and seed it with the priority generator
a.dispatcher = new PriorityDispatcher("foo", gen)
a.start // Start the Actor
}).withDispatcher(new PriorityDispatcher("foo", gen))) // We create a new Priority dispatcher and seed it with the priority generator
a.dispatcher.suspend(a) // Suspending the actor so it doesn't start to treat the messages before we have enqueued all of them :-)

View file

@ -214,7 +214,7 @@ Here is the API and how to use it from within an 'Actor':
self.unlink(actorRef)
// link first, then start actor
self.link(actorRef).start()
self.link(actorRef)
// spawns (creates and starts) actors
self.spawn[MyActor]
@ -368,7 +368,7 @@ Here is an example:
victimActorRef, maxNrOfRetries, withinTimeRange, lastExceptionCausingRestart) =>
... // handle the error situation
}
}).start()
})
You will also get this log warning similar to this:

View file

@ -207,7 +207,7 @@ Finally, bind the *handleHttpRequest* function of the *Endpoint* trait to the ac
def hook(uri: String): Boolean = ((uri == ProvideSameActor) || (uri == ProvideNewActor))
def provide(uri: String): ActorRef = {
if (uri == ProvideSameActor) same
else actorOf[BoringActor].start()
else actorOf[BoringActor]
}
//
@ -236,7 +236,7 @@ Finally, bind the *handleHttpRequest* function of the *Endpoint* trait to the ac
//
// this will be our "same" actor provided with ProvideSameActor endpoint is hit
//
lazy val same = actorOf[BoringActor].start()
lazy val same = actorOf[BoringActor]
}
Handling requests
@ -333,10 +333,10 @@ As noted above, hook functions are non-exclusive. This means multiple actors can
// Try with/without a header named "Test-Token"
// Try with/without a form parameter named "Data"
def hookMultiActionA(uri: String): Boolean = uri startsWith Multi
def provideMultiActionA(uri: String): ActorRef = actorOf(new ActionAActor(complete)).start()
def provideMultiActionA(uri: String): ActorRef = actorOf(new ActionAActor(complete))
def hookMultiActionB(uri: String): Boolean = uri startsWith Multi
def provideMultiActionB(uri: String): ActorRef = actorOf(new ActionBActor(complete)).start()
def provideMultiActionB(uri: String): ActorRef = actorOf(new ActionBActor(complete))
//
// this is where you want attach your endpoint hooks
@ -365,7 +365,7 @@ As noted above, hook functions are non-exclusive. This means multiple actors can
//
// this guy completes requests after other actions have occurred
//
lazy val complete = actorOf[ActionCompleteActor].start()
lazy val complete = actorOf[ActionCompleteActor]
}
class ActionAActor(complete:ActorRef) extends Actor {

View file

@ -36,7 +36,7 @@ Here is how to start up the RemoteNode and specify the hostname and port in the
import akka.actor.Actor._
remote.start()
remote
// Specify the classloader to use to load the remote class (actor)
remote.start(classLoader)
@ -592,7 +592,7 @@ So a simple listener actor can look like this:
case RemoteClientWriteFailed(request, cause, client, address) => //... act upon write failure
case _ => // ignore other
}
}).start()
})
Registration and de-registration can be done like this:
@ -650,7 +650,7 @@ So a simple listener actor can look like this:
case RemoteServerClientClosed(server, clientAddress) => //... act upon client connection close
case RemoteServerWriteFailed(request, cause, server, clientAddress) => //... act upon server write failure
}
}).start()
})
Registration and de-registration can be done like this:

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) } }).start()
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } }).start()
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
val ponger = actorOf(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) } }).start()
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } }).start()
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
val ponger = actorOf(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].start()
val d = actorOf[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) } }).start()
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } }).start()
val pinger = actorOf(new Actor { def receive = { case x => println("Pinger: " + x) } })
val ponger = actorOf(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) } }).start()
val ponger = actorOf(new Actor { def receive = { case x => println("Ponger: " + x) } }).start()
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 seq = new CyclicIterator[ActorRef](List(pinger,ponger))
}
//Create an instance of our loadbalancer, and start it
val d = actorOf[MyLoadBalancer].start()
val d = actorOf[MyLoadBalancer]
d ! Pong //Prints "Pinger: Pong"
d ! Pong //Prints "Ponger: Pong"

View file

@ -95,13 +95,12 @@ Step 3: Import the type class module definition and serialize / de-serialize::
import akka.serialization.ActorSerialization._
import BinaryFormatMyActor._
val actor1 = actorOf[MyActor].start()
val actor1 = actorOf[MyActor]
(actor1 ? "hello").as[String].getOrElse("_") should equal("world 1")
(actor1 ? "hello").as[String].getOrElse("_") should equal("world 2")
val bytes = toBinary(actor1)
val actor2 = fromBinary(bytes)
actor2.start()
(actor2 ? "hello").as[String].getOrElse("_") should equal("world 3")
}
@ -137,13 +136,12 @@ and use it for serialization::
import akka.serialization.ActorSerialization._
import BinaryFormatMyStatelessActor._
val actor1 = actorOf[MyStatelessActor].start()
val actor1 = actorOf[MyStatelessActor]
(actor1 ? "hello").as[String].getOrElse("_") should equal("world")
(actor1 ? "hello").as[String].getOrElse("_") should equal("world")
val bytes = toBinary(actor1)
val actor2 = fromBinary(bytes)
actor2.start()
(actor2 ? "hello").as[String].getOrElse("_") should equal("world")
}
@ -188,13 +186,12 @@ and serialize / de-serialize::
import akka.serialization.ActorSerialization._
import BinaryFormatMyJavaSerializableActor._
val actor1 = actorOf[MyJavaSerializableActor].start()
val actor1 = actorOf[MyJavaSerializableActor]
(actor1 ? "hello").as[String].getOrElse("_") should equal("world 1")
(actor1 ? "hello").as[String].getOrElse("_") should equal("world 2")
val bytes = toBinary(actor1)
val actor2 = fromBinary(bytes)
actor2.start()
(actor2 ? "hello").as[String].getOrElse("_") should equal("world 3")
}

View file

@ -370,7 +370,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).start()
val transferer = Actor.actorOf(new Transferer)
transferer ! Transfer(account1, account2, 500.0)
// INFO Transferer: not enough money - retrying
@ -426,7 +426,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).start()
val brancher = Actor.actorOf(new Brancher)
brancher ! Branch(ref1, ref2, 1)
// INFO Brancher: not enough on left - retrying

View file

@ -105,7 +105,7 @@ and in addition allows access to the internal state::
when (2) {
case Ev("back") => goto(1) using "back"
}
}).start()
})
assert (fsm.stateName == 1)
assert (fsm.stateData == "")
@ -236,7 +236,7 @@ common task easy:
"send back messages unchanged" in {
val echo = Actor.actorOf[EchoActor].start()
val echo = Actor.actorOf[EchoActor]
echo ! "hello world"
expectMsg("hello world")
@ -474,7 +474,7 @@ You would then need to make an implicit available in locally confined scopes
which need it, e.g. different test cases. If this cannot be done, you will need
to resort to explicitly specifying the sender reference::
val actor = actorOf[MyWorker].start()
val actor = actorOf[MyWorker]
actor.!(msg)(testActor)
Using Multiple Probe Actors
@ -503,7 +503,7 @@ using a small example::
val probe1 = TestProbe()
val probe2 = TestProbe()
val actor = Actor.actorOf[MyDoubleEcho].start()
val actor = Actor.actorOf[MyDoubleEcho]
actor ! (probe1.ref, probe2.ref)
actor ! "hello"
probe1.expectMsg(50 millis, "hello")
@ -550,8 +550,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)).start()
val dest = Actor.actorOf[Destination].start()
val source = Actor.actorOf(new Source(probe))
val dest = Actor.actorOf[Destination]
source ! "start"
probe.expectMsg("work")
probe.forward(dest)
@ -610,9 +610,7 @@ or from the client code
.. code-block:: scala
val ref = Actor.actorOf[MyActor]
ref.dispatcher = CallingThreadDispatcher.global
ref.start()
val ref = Actor.actorOf(Props[MyActor].withDispatcher(CallingThreadDispatcher.global))
As the :class:`CallingThreadDispatcher` does not have any configurable state,
you may always use the (lazily) preallocated one as shown in the examples.

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