Merge branch 'master' of github.com:jboner/akka

Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
Jonas Bonér 2011-08-29 14:26:24 +02:00
commit 5e290ec393
127 changed files with 2412 additions and 1682 deletions

View file

@ -1,5 +1,6 @@
package akka.dispatch;
import akka.actor.Timeout;
import org.junit.Test;
import static org.junit.Assert.*;
import java.util.concurrent.Callable;
@ -11,6 +12,7 @@ import java.util.concurrent.TimeUnit;
import akka.japi.Function;
import akka.japi.Function2;
import akka.japi.Procedure;
import akka.japi.Option;
import scala.Some;
import scala.Right;
import static akka.dispatch.Futures.*;
@ -231,4 +233,25 @@ public class JavaFutureTests {
assertEquals(result.get(), expectedStrings);
}
@Test public void findForJavaApiMustWork() {
LinkedList<Future<Integer>> listFutures = new LinkedList<Future<Integer>>();
for (int i = 0; i < 10; i++) {
final Integer fi = i;
listFutures.add(future(new Callable<Integer>() {
public Integer call() {
return fi;
}
}));
}
final Integer expect = 5;
Future<Option<Integer>> f = Futures.find(listFutures, new Function<Integer,Boolean>() {
public Boolean apply(Integer i) {
return i == 5;
}
}, Timeout.getDefault());
final Integer got = f.get().get();
assertEquals(expect, got);
}
}

View file

@ -27,9 +27,7 @@ object ActorFireForgetRequestReplySpec {
}
}
class CrashingTemporaryActor extends Actor {
self.lifeCycle = Temporary
class CrashingActor extends Actor {
def receive = {
case "Die"
state.finished.await
@ -86,7 +84,7 @@ class ActorFireForgetRequestReplySpec extends WordSpec with MustMatchers with Be
"should shutdown crashed temporary actor" in {
filterEvents(EventFilter[Exception]("Expected")) {
val actor = actorOf[CrashingTemporaryActor].start()
val actor = actorOf(Props[CrashingActor].withLifeCycle(Temporary)).start()
actor.isRunning must be(true)
actor ! "Die"
state.finished.await

View file

@ -15,6 +15,7 @@ import akka.dispatch.Future
import java.util.concurrent.{ TimeUnit, CountDownLatch }
import java.lang.IllegalStateException
import akka.util.ReflectiveAccess
import akka.actor.Actor.actorOf
object ActorRefSpec {
@ -26,11 +27,11 @@ object ActorRefSpec {
def receive = {
case "complexRequest" {
replyTo = self.channel
val worker = Actor.actorOf[WorkerActor].start()
val worker = actorOf(Props[WorkerActor])
worker ! "work"
}
case "complexRequest2"
val worker = Actor.actorOf[WorkerActor].start()
val worker = actorOf(Props[WorkerActor])
worker ! self.channel
case "workDone" replyTo ! "complexReply"
case "simpleRequest" self.reply("simpleReply")
@ -122,7 +123,7 @@ class ActorRefSpec extends WordSpec with MustMatchers {
}
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new Actor {
actorOf(new Actor {
val nested = new Actor { def receive = { case _ } }
def receive = { case _ }
}).start()
@ -133,43 +134,43 @@ class ActorRefSpec extends WordSpec with MustMatchers {
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new FailingOuterActor(Actor.actorOf(new InnerActor).start)).start()
actorOf(new FailingOuterActor(actorOf(new InnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new OuterActor(Actor.actorOf(new FailingInnerActor).start)).start()
actorOf(new OuterActor(actorOf(new FailingInnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new FailingInheritingOuterActor(Actor.actorOf(new InnerActor).start)).start()
actorOf(new FailingInheritingOuterActor(actorOf(new InnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new FailingOuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new FailingInheritingOuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new FailingInheritingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new FailingInheritingOuterActor(Actor.actorOf(new FailingInnerActor).start)).start()
actorOf(new FailingInheritingOuterActor(actorOf(new FailingInnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new OuterActor(Actor.actorOf(new InnerActor {
actorOf(new OuterActor(actorOf(new InnerActor {
val a = new InnerActor
}).start)).start()
}
@ -177,32 +178,32 @@ class ActorRefSpec extends WordSpec with MustMatchers {
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new FailingOuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new OuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start()
actorOf(new OuterActor(actorOf(new FailingInheritingInnerActor).start)).start()
}
refStackMustBeEmpty
intercept[akka.actor.ActorInitializationException] {
Actor.actorOf(new OuterActor(Actor.actorOf({ new InnerActor; new InnerActor }).start)).start()
actorOf(new OuterActor(actorOf({ new InnerActor; new InnerActor }).start)).start()
}
refStackMustBeEmpty
(intercept[java.lang.IllegalStateException] {
Actor.actorOf(new OuterActor(Actor.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 }).start)).start()
}).getMessage must be === "Ur state be b0rked"
refStackMustBeEmpty
}
"be serializable using Java Serialization on local node" in {
val a = Actor.actorOf[InnerActor].start
val a = actorOf[InnerActor].start
import java.io._
@ -225,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 = Actor.actorOf[InnerActor].start
val a = actorOf[InnerActor].start
val inetAddress = ReflectiveAccess.RemoteModule.configDefaultAddress
@ -255,8 +256,8 @@ class ActorRefSpec extends WordSpec with MustMatchers {
}
"support nested actorOfs" in {
val a = Actor.actorOf(new Actor {
val nested = Actor.actorOf(new Actor { def receive = { case _ } }).start()
val a = actorOf(new Actor {
val nested = actorOf(new Actor { def receive = { case _ } }).start()
def receive = { case _ self reply nested }
}).start()
@ -267,7 +268,7 @@ class ActorRefSpec extends WordSpec with MustMatchers {
}
"support advanced nested actorOfs" in {
val a = Actor.actorOf(new OuterActor(Actor.actorOf(new InnerActor).start)).start
val a = actorOf(Props(new OuterActor(actorOf(Props(new InnerActor)))))
val inner = (a ? "innerself").as[Any].get
(a ? a).as[ActorRef].get must be(a)
@ -278,8 +279,8 @@ class ActorRefSpec extends WordSpec with MustMatchers {
}
"support reply via channel" in {
val serverRef = Actor.actorOf[ReplyActor].start()
val clientRef = Actor.actorOf(new SenderActor(serverRef)).start()
val serverRef = actorOf(Props[ReplyActor])
val clientRef = actorOf(Props(new SenderActor(serverRef)))
clientRef ! "complex"
clientRef ! "simple"
@ -302,16 +303,14 @@ class ActorRefSpec extends WordSpec with MustMatchers {
}
"stop when sent a poison pill" in {
val ref = Actor.actorOf(
new Actor {
def receive = {
case 5 self tryReply "five"
case null self tryReply "null"
}
}).start()
val timeout = Timeout(20000)
val ref = actorOf(Props(self {
case 5 self tryReply "five"
case null self tryReply "null"
}))
val ffive = (ref ? 5).mapTo[String]
val fnull = (ref ? null).mapTo[String]
val ffive = (ref ? (5, timeout)).mapTo[String]
val fnull = (ref ? (null, timeout)).mapTo[String]
intercept[ActorKilledException] {
(ref ? PoisonPill).get
@ -329,20 +328,17 @@ class ActorRefSpec extends WordSpec with MustMatchers {
filterException[ActorKilledException] {
val latch = new CountDownLatch(2)
val boss = Actor.actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), scala.Some(2), scala.Some(1000))
val boss = actorOf(Props(new Actor {
val ref = Actor.actorOf(
new Actor {
val ref = actorOf(
Props(new Actor {
def receive = { case _ }
override def preRestart(reason: Throwable, msg: Option[Any]) = latch.countDown()
override def postRestart(reason: Throwable) = latch.countDown()
}).start()
self link ref
}).withSupervisor(self))
protected def receive = { case "sendKill" ref ! Kill }
}).start()
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000)))
boss ! "sendKill"
latch.await(5, TimeUnit.SECONDS) must be === true

View file

@ -59,7 +59,6 @@ object ActorRestartSpec {
}
class Supervisor extends Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 5000)
def receive = {
case _
}
@ -79,21 +78,24 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo
}
private var toStop = new java.util.concurrent.ConcurrentSkipListSet[ActorRef]
private def newActor(f: Actor): ActorRef = {
val ref = actorOf(f)
private def collect(f: ActorRef): ActorRef = {
val ref = f
toStop add ref
ref.start()
}
private def createSupervisor =
actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
val expectedEvents = Seq(EventFilter[ActorKilledException], EventFilter[IllegalActorStateException]("expected"))
"An Actor restart" must {
"invoke preRestart, preStart, postRestart" in {
filterEvents(expectedEvents) {
val actor = newActor(new Restarter(testActor))
val actor = collect(actorOf(new Restarter(testActor)))
expectMsg(1 second, ("preStart", 1))
val supervisor = newActor(new Supervisor)
val supervisor = collect(createSupervisor)
supervisor link actor
actor ! Kill
within(1 second) {
@ -107,9 +109,9 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo
"support creation of nested actors in freshInstance()" in {
filterEvents(expectedEvents) {
val actor = newActor(new Restarter(testActor))
val actor = collect(actorOf(new Restarter(testActor)))
expectMsg(1 second, ("preStart", 1))
val supervisor = newActor(new Supervisor)
val supervisor = collect(createSupervisor)
supervisor link actor
actor ! Nested
actor ! Kill
@ -128,9 +130,9 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo
"use freshInstance() if available" in {
filterEvents(expectedEvents) {
val actor = newActor(new Restarter(testActor))
val actor = collect(actorOf(new Restarter(testActor)))
expectMsg(1 second, ("preStart", 1))
val supervisor = newActor(new Supervisor)
val supervisor = collect(createSupervisor)
supervisor link actor
actor ! 42
actor ! Handover
@ -148,9 +150,9 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo
"fall back to default factory if freshInstance() fails" in {
filterEvents(expectedEvents) {
val actor = newActor(new Restarter(testActor))
val actor = collect(actorOf(new Restarter(testActor)))
expectMsg(1 second, ("preStart", 1))
val supervisor = newActor(new Supervisor)
val supervisor = collect(createSupervisor)
supervisor link actor
actor ! 42
actor ! Fail

View file

@ -15,38 +15,42 @@ class ActorTimeoutSpec
with MustMatchers
with TestKit {
val echo = Actor.actorOf(new Actor {
def actorWithTimeout(t: Timeout): ActorRef = Actor.actorOf(Props(creator = () new Actor {
def receive = {
case x
}
}).start()
}, timeout = t)).start()
val testTimeout = if (Timeout.default.duration < 400.millis) 500 millis else 100 millis
override def afterAll { echo.stop() }
"An Actor-based Future" must {
"use the global default timeout if no implicit in scope" in {
echo.timeout = 12
within((Actor.TIMEOUT - 100).millis, (Actor.TIMEOUT + 400).millis) {
val f = echo ? "hallo"
intercept[FutureTimeoutException] { f.await }
val echo = actorWithTimeout(Timeout(12))
try {
val f = echo ? "hallo"
intercept[FutureTimeoutException] { f.await }
} finally { echo.stop }
}
}
"use implicitly supplied timeout" in {
implicit val timeout = Timeout(testTimeout)
within(testTimeout - 100.millis, testTimeout + 300.millis) {
val f = (echo ? "hallo").mapTo[String]
intercept[FutureTimeoutException] { f.await }
f.value must be(None)
val echo = actorWithTimeout(Props.defaultTimeout)
try {
val f = (echo ? "hallo").mapTo[String]
intercept[FutureTimeoutException] { f.await }
f.value must be(None)
} finally { echo.stop }
}
}
"use explicitly supplied timeout" in {
within(testTimeout - 100.millis, testTimeout + 300.millis) {
(echo.?("hallo", testTimeout)).as[String] must be(None)
val echo = actorWithTimeout(Props.defaultTimeout)
try { (echo.?("hallo", testTimeout)).as[String] must be(None) } finally { echo.stop }
}
}
}

View file

@ -48,7 +48,7 @@ class ClusterSpec extends WordSpec with MustMatchers {
getInt("akka.cluster.server.max-total-memory-size") must equal(Some(0))
//akka.cluster.client
getBool("akka.cluster.client.buffering.retry-message-send-on-failure") must equal(Some(true))
getBool("akka.cluster.client.buffering.retry-message-send-on-failure") must equal(Some(false))
getInt("akka.cluster.client.buffering.capacity") must equal(Some(-1))
getInt("akka.cluster.client.reconnect-delay") must equal(Some(5))
getInt("akka.cluster.client.read-timeout") must equal(Some(3600))

View file

@ -22,7 +22,7 @@ class DeployerSpec extends WordSpec with MustMatchers {
LeastCPU,
Clustered(
Vector(Node("node1")),
ReplicationFactor(3),
new ReplicationFactor(3),
Replication(
TransactionLog,
WriteThrough)))))

View file

@ -70,7 +70,7 @@ class FSMTimingSpec extends WordSpec with MustMatchers with TestKit {
fsm ! Tick
expectMsg(100 millis, Tick)
Thread.sleep(200)
fsm.dispatcher resume fsm
resume(fsm)
expectMsg(100 millis, Transition(fsm, TestCancelStateTimerInNamedTimerMessage, TestCancelStateTimerInNamedTimerMessage2))
fsm ! Cancel
within(100 millis) {
@ -119,6 +119,16 @@ class FSMTimingSpec extends WordSpec with MustMatchers with TestKit {
object FSMTimingSpec {
def suspend(actorRef: ActorRef): Unit = actorRef match {
case l: LocalActorRef l.dispatcher.suspend(l)
case _
}
def resume(actorRef: ActorRef): Unit = actorRef match {
case l: LocalActorRef l.dispatcher.resume(l)
case _
}
trait State
case object Initial extends State
case object TestStateTimeout extends State
@ -189,7 +199,7 @@ object FSMTimingSpec {
when(TestCancelStateTimerInNamedTimerMessage) {
// FSM is suspended after processing this message and resumed 200ms later
case Ev(Tick)
self.dispatcher suspend self
suspend(self)
setTimer("named", Tock, 10 millis, false)
stay forMax (100 millis) replying Tick
case Ev(Tock)

View file

@ -17,7 +17,6 @@ import FSM._
object FSMTransitionSpec {
class Supervisor extends Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, None)
def receive = { case _ }
}
@ -63,7 +62,7 @@ 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 sup = Actor.actorOf[Supervisor].start()
val sup = Actor.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
sup link fsm
within(300 millis) {
fsm ! SubscribeTransitionCallBack(forward)
@ -77,7 +76,7 @@ class FSMTransitionSpec extends WordSpec with MustMatchers with TestKit {
"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[Supervisor].start()
val sup = Actor.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
sup link fsm
within(300 millis) {
filterEvents(EventFilter.custom {

View file

@ -11,49 +11,22 @@ import akka.testkit._
import akka.util.duration._
import Actor._
import akka.util.Duration
object ForwardActorSpec {
object ForwardState {
var sender: Option[ActorRef] = None
}
val ExpectedMessage = "FOO"
class ReceiverActor extends Actor {
val latch = TestLatch()
def receive = {
case "SendBang" {
ForwardState.sender = self.sender
latch.countDown()
}
case "SendBangBang" self.reply("SendBangBang")
}
}
def createForwardingChain(): ActorRef = {
val replier = actorOf(new Actor {
def receive = { case x self reply x }
}).start()
class ForwardActor extends Actor {
val receiverActor = actorOf[ReceiverActor]
receiverActor.start()
def receive = {
case "SendBang" receiverActor.forward("SendBang")
case "SendBangBang" receiverActor.forward("SendBangBang")
}
}
def mkforwarder(forwardTo: ActorRef) = actorOf(
new Actor {
def receive = { case x forwardTo forward x }
}).start()
class BangSenderActor extends Actor {
val forwardActor = actorOf[ForwardActor]
forwardActor.start()
forwardActor ! "SendBang"
def receive = {
case _ {}
}
}
class BangBangSenderActor extends Actor {
val latch = TestLatch()
val forwardActor = actorOf[ForwardActor]
forwardActor.start()
forwardActor ? "SendBangBang" onComplete { _ latch.countDown() }
def receive = {
case _ {}
}
mkforwarder(mkforwarder(mkforwarder(replier)))
}
}
@ -61,23 +34,21 @@ class ForwardActorSpec extends WordSpec with MustMatchers {
import ForwardActorSpec._
"A Forward Actor" must {
"forward actor reference when invoking forward on bang" in {
val senderActor = actorOf[BangSenderActor]
val latch = senderActor.actor.asInstanceOf[BangSenderActor]
.forwardActor.actor.asInstanceOf[ForwardActor]
.receiverActor.actor.asInstanceOf[ReceiverActor]
.latch
senderActor.start()
latch.await
ForwardState.sender must not be (null)
senderActor.toString must be(ForwardState.sender.get.toString)
val latch = new TestLatch(1)
val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } }).start()
val chain = createForwardingChain()
chain.tell(ExpectedMessage, replyTo)
latch.await(Duration(5, "s")) must be === true
}
"forward actor reference when invoking forward on bang bang" in {
val senderActor = actorOf[BangBangSenderActor]
senderActor.start()
val latch = senderActor.actor.asInstanceOf[BangBangSenderActor].latch
latch.await
val chain = createForwardingChain()
chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage
}
}
}

View file

@ -34,7 +34,8 @@ object IOActorSpec {
})
def receive = {
case msg: NewClient self startLink createWorker forward msg
case msg: NewClient
self.link(createWorker).start() forward msg
}
}
@ -103,7 +104,7 @@ object IOActorSpec {
})
def receive = {
case msg: NewClient self startLink createWorker forward msg
case msg: NewClient self.link(createWorker).start() forward msg
case ('set, key: String, value: ByteString)
kvs += (key -> value)
self tryReply (())

View file

@ -131,12 +131,9 @@ class LoggingReceiveSpec
"log LifeCycle changes if requested" in {
within(2 seconds) {
val supervisor = TestActorRef(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 5000)
def receive = {
case _
}
}).start()
val supervisor = TestActorRef(Props(new Actor {
def receive = { case _ }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
val f = Actor.getClass.getDeclaredField("debugLifecycle")
f.setAccessible(true)
f.setBoolean(Actor, true)

View file

@ -34,10 +34,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def slaveShouldStayDeadAfterMaxRestartsWithinTimeRange = {
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(1000))
val boss = actorOf(Props(new Actor {
protected def receive = { case _ () }
}).start()
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000)))
val restartLatch = new StandardLatch
val secondRestartLatch = new StandardLatch
@ -61,7 +60,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
stopLatch.open
}
})
boss.startLink(slave)
boss.link(slave).start()
slave ! Ping
slave ! Crash
@ -88,10 +87,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def slaveShouldBeImmortalWithoutMaxRestartsAndTimeRange = {
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, None)
protected def receive = { case _ () }
}).start()
val boss = actorOf(Props(new Actor {
def receive = { case _ () }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
val countDownLatch = new CountDownLatch(100)
@ -106,7 +104,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
}
})
boss.startLink(slave)
boss.link(slave).start()
(1 to 100) foreach { _ slave ! Crash }
assert(countDownLatch.await(120, TimeUnit.SECONDS))
assert(slave.isRunning)
@ -115,10 +113,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def slaveShouldRestartAfterNumberOfCrashesNotWithinTimeRange = {
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(500))
protected def receive = { case _ () }
}).start()
val boss = actorOf(Props(new Actor {
def receive = { case _ () }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 500)))
val restartLatch = new StandardLatch
val secondRestartLatch = new StandardLatch
@ -148,7 +145,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
}
}
})
boss.startLink(slave)
boss.link(slave).start()
slave ! Ping
slave ! Crash
@ -176,10 +173,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def slaveShouldNotRestartAfterMaxRetries = {
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), None)
protected def receive = { case _ () }
}).start()
val boss = actorOf(Props(new Actor {
def receive = { case _ () }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), Some(2), None)))
val restartLatch = new StandardLatch
val secondRestartLatch = new StandardLatch
@ -203,7 +199,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
stopLatch.open
}
})
boss.startLink(slave)
boss.link(slave).start()
slave ! Ping
slave ! Crash
@ -235,12 +231,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
val restartLatch, stopLatch, maxNoOfRestartsLatch = new StandardLatch
val countDownLatch = new CountDownLatch(2)
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))
protected def receive = {
case m: MaximumNumberOfRestartsWithinTimeRangeReached maxNoOfRestartsLatch.open
}
}).start()
val boss = actorOf(Props(new Actor {
def receive = { case m: MaximumNumberOfRestartsWithinTimeRangeReached maxNoOfRestartsLatch.open }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))))
val slave = actorOf(new Actor {
@ -257,7 +250,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll {
stopLatch.open
}
})
boss.startLink(slave)
boss.link(slave).start()
slave ! Ping
slave ! Crash

View file

@ -36,18 +36,16 @@ class SupervisorHierarchySpec extends JUnitSuite {
val workerTwo = actorOf(new CountDownActor(countDown))
val workerThree = actorOf(new CountDownActor(countDown))
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 1000)
val boss = actorOf(Props(new Actor {
protected def receive = { case _ () }
}).start()
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 1000)))
val manager = actorOf(new CountDownActor(countDown))
boss.startLink(manager)
boss.link(manager).start()
manager.startLink(workerOne)
manager.startLink(workerTwo)
manager.startLink(workerThree)
manager.link(workerOne).start()
manager.link(workerTwo).start()
manager.link(workerThree).start()
workerOne ! Death(workerOne, new FireWorkerException("Fire the worker!"))
@ -61,14 +59,12 @@ class SupervisorHierarchySpec extends JUnitSuite {
def supervisorShouldReceiveNotificationMessageWhenMaximumNumberOfRestartsWithinTimeRangeIsReached = {
val countDown = new CountDownLatch(2)
val crasher = actorOf(new CountDownActor(countDown))
val boss = actorOf(new Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 1, 5000)
val boss = actorOf(Props(new Actor {
protected def receive = {
case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _)
countDown.countDown()
case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) countDown.countDown()
}
}).start()
boss.startLink(crasher)
}).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!"))

View file

@ -5,10 +5,10 @@ package akka.actor
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.dispatch.Dispatchers
import akka.config.Supervision.{ SupervisorConfig, OneForOneStrategy, Supervise, Permanent }
import java.util.concurrent.CountDownLatch
import akka.testkit.{ filterEvents, EventFilter }
import akka.dispatch.{ PinnedDispatcher, Dispatchers }
class SupervisorMiscSpec extends WordSpec with MustMatchers {
"A Supervisor" should {
@ -17,45 +17,41 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers {
filterEvents(EventFilter[Exception]("killed")) {
val countDownLatch = new CountDownLatch(4)
val actor1 = Actor.actorOf(new Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
val actor1 = Actor.actorOf(Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
protected def receive = {
case "kill" throw new Exception("killed")
case _ println("received unknown message")
}
}).start()
}).withDispatcher(new PinnedDispatcher()))
val actor2 = Actor.actorOf(new Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
val actor2 = Actor.actorOf(Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
protected def receive = {
case "kill" throw new Exception("killed")
case _ println("received unknown message")
}
}).start()
}).withDispatcher(new PinnedDispatcher()))
val actor3 = Actor.actorOf(new Actor {
self.dispatcher = Dispatchers.newDispatcher("test").build
val actor3 = Actor.actorOf(Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
protected def receive = {
case "kill" throw new Exception("killed")
case _ println("received unknown message")
}
}).start()
}).withDispatcher(Dispatchers.newDispatcher("test").build))
val actor4 = Actor.actorOf(new Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
val actor4 = Actor.actorOf(Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
protected def receive = {
case "kill" throw new Exception("killed")
case _ println("received unknown message")
}
}).start()
}).withDispatcher(new PinnedDispatcher()))
val sup = Supervisor(
SupervisorConfig(

View file

@ -55,15 +55,10 @@ object SupervisorSpec {
}
}
class TemporaryActor extends PingPongActor {
self.lifeCycle = Temporary
}
class Master extends Actor {
self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, (1 second).dilated.toMillis.toInt)
val temp = {
val a = actorOf[TemporaryActor]
val a = actorOf(Props[PingPongActor].withLifeCycle(Temporary))
self link a
a.start
}
@ -79,7 +74,7 @@ object SupervisorSpec {
// =====================================================
def temporaryActorAllForOne = {
val temporaryActor = actorOf[TemporaryActor].start()
val temporaryActor = actorOf(Props[PingPongActor].withLifeCycle(Temporary))
val supervisor = Supervisor(
SupervisorConfig(
@ -226,7 +221,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
"A supervisor" must {
"not restart programmatically linked temporary actor" in {
val master = actorOf[Master].start()
val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, (1 second).dilated.toMillis.toInt)))
intercept[RuntimeException] {
(master.?(Die, TimeoutMillis)).get
@ -369,8 +364,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
"must attempt restart when exception during restart" in {
val inits = new AtomicInteger(0)
val dyingActor = actorOf(new Actor {
self.lifeCycle = Permanent
val dyingActor = actorOf(Props(new Actor {
inits.incrementAndGet
if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!")
@ -379,7 +373,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach
case Ping self.tryReply(PongMessage)
case Die throw new Exception("expected")
}
})
}))
val supervisor =
Supervisor(

View file

@ -17,9 +17,7 @@ class SupervisorTreeSpec extends WordSpec with MustMatchers {
var log = ""
case object Die
class Chainer(a: Option[ActorRef] = None) extends Actor {
self.lifeCycle = Permanent
self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 3, 1000)
class Chainer(a: Option[ActorRef]) extends Actor {
a.foreach(self.link(_))
def receive = {
@ -37,9 +35,10 @@ class SupervisorTreeSpec extends WordSpec with MustMatchers {
filterException[Exception] {
log = "INIT"
val lastActor = actorOf(new Chainer, "lastActor").start
val middleActor = actorOf(new Chainer(Some(lastActor)), "middleActor").start
val headActor = actorOf(new Chainer(Some(middleActor)), "headActor").start
val p = Props.default.withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000))
val lastActor = actorOf(p.withCreator(new Chainer(None)), "lastActor")
val middleActor = actorOf(p.withCreator(new Chainer(Some(lastActor))), "middleActor")
val headActor = actorOf(p.withCreator(new Chainer(Some(middleActor))), "headActor")
middleActor ! Die
sleepFor(500 millis)

View file

@ -12,9 +12,9 @@ import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit }
import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor
import akka.util.Switch
import akka.actor.{ ActorKilledException, PoisonPill, ActorRef, Actor }
import java.rmi.RemoteException
import org.junit.{ After, Test }
import akka.actor._
object ActorModelSpec {
@ -49,11 +49,11 @@ object ActorModelSpec {
val Ping = "Ping"
val Pong = "Pong"
class DispatcherActor(dispatcher: MessageDispatcherInterceptor) extends Actor {
self.dispatcher = dispatcher.asInstanceOf[MessageDispatcher]
class DispatcherActor extends Actor {
private val busy = new Switch(false)
def dispatcher = self.dispatcher.asInstanceOf[MessageDispatcherInterceptor]
def ack {
if (!busy.switchOn()) {
throw new Exception("isolation violated")
@ -103,22 +103,22 @@ object ActorModelSpec {
stats.get(actorRef)
}
abstract override def suspend(actorRef: ActorRef) {
abstract override def suspend(actorRef: LocalActorRef) {
super.suspend(actorRef)
getStats(actorRef).suspensions.incrementAndGet()
}
abstract override def resume(actorRef: ActorRef) {
abstract override def resume(actorRef: LocalActorRef) {
super.resume(actorRef)
getStats(actorRef).resumes.incrementAndGet()
}
private[akka] abstract override def register(actorRef: ActorRef) {
private[akka] abstract override def register(actorRef: LocalActorRef) {
super.register(actorRef)
getStats(actorRef).registers.incrementAndGet()
}
private[akka] abstract override def unregister(actorRef: ActorRef) {
private[akka] abstract override def unregister(actorRef: LocalActorRef) {
super.unregister(actorRef)
getStats(actorRef).unregisters.incrementAndGet()
}
@ -183,7 +183,7 @@ object ActorModelSpec {
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get()) {
val stats = statsFor(actorRef, if (dispatcher eq null) actorRef.dispatcher else dispatcher)
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[SelfActorRef].dispatcher))
assert(stats.suspensions.get() === suspensions, "Suspensions")
assert(stats.resumes.get() === resumes, "Resumes")
assert(stats.registers.get() === registers, "Registers")
@ -207,7 +207,7 @@ object ActorModelSpec {
false
}
def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(new DispatcherActor(d))
def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(Props[DispatcherActor].withDispatcher(d))
}
abstract class ActorModelSpec extends JUnitSuite {
@ -219,8 +219,8 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldDynamicallyHandleItsOwnLifeCycle {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor
assertDispatcher(dispatcher)(starts = 0, stops = 0)
val a = newTestActor
a.start()
assertDispatcher(dispatcher)(starts = 1, stops = 0)
a.stop()
@ -346,7 +346,7 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldNotProcessMessagesForASuspendedActor {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start()
val a = newTestActor.start().asInstanceOf[LocalActorRef]
val done = new CountDownLatch(1)
dispatcher.suspend(a)
a ! CountDown(done)
@ -384,7 +384,7 @@ abstract class ActorModelSpec extends JUnitSuite {
@Test
def dispatcherShouldCompleteAllUncompletedSenderFuturesOnDeregister {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor.start()
val a = newTestActor.start().asInstanceOf[LocalActorRef]
dispatcher.suspend(a)
val f1: Future[String] = a ? Reply("foo") mapTo manifest[String]
val stopped = a ? PoisonPill

View file

@ -6,9 +6,9 @@ import org.scalatest.junit.JUnitSuite
import org.junit.Test
import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.actor.{ IllegalActorStateException, Actor }
import Actor._
import akka.actor.Actor._
import akka.dispatch.{ MessageQueue, Dispatchers }
import akka.actor.{ LocalActorRef, IllegalActorStateException, Actor, Props }
object BalancingDispatcherSpec {
@ -17,7 +17,6 @@ object BalancingDispatcherSpec {
val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer()
class DelayableActor(delay: Int, finishedCounter: CountDownLatch) extends Actor {
self.dispatcher = delayableActorDispatcher
@volatile
var invocationCount = 0
@ -31,17 +30,14 @@ object BalancingDispatcherSpec {
}
class FirstActor extends Actor {
self.dispatcher = sharedActorDispatcher
def receive = { case _ {} }
}
class SecondActor extends Actor {
self.dispatcher = sharedActorDispatcher
def receive = { case _ {} }
}
class ParentActor extends Actor {
self.dispatcher = parentActorDispatcher
def receive = { case _ {} }
}
@ -59,8 +55,8 @@ class BalancingDispatcherSpec extends JUnitSuite with MustMatchers {
def fastActorShouldStealWorkFromSlowActor {
val finishedCounter = new CountDownLatch(110)
val slow = actorOf(new DelayableActor(50, finishedCounter), "slow").start
val fast = actorOf(new DelayableActor(10, finishedCounter), "fast").start
val slow = actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher), "slow").asInstanceOf[LocalActorRef]
val fast = actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher), "fast").asInstanceOf[LocalActorRef]
var sentToFast = 0
@ -86,32 +82,32 @@ class BalancingDispatcherSpec extends JUnitSuite with MustMatchers {
finishedCounter.await(5, TimeUnit.SECONDS)
fast.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true)
slow.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true)
fast.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
fast.actor.asInstanceOf[DelayableActor].invocationCount must be >
(slow.actor.asInstanceOf[DelayableActor].invocationCount)
fast.actorInstance.get().asInstanceOf[DelayableActor].invocationCount must be > sentToFast
fast.actorInstance.get().asInstanceOf[DelayableActor].invocationCount must be >
(slow.actorInstance.get().asInstanceOf[DelayableActor].invocationCount)
slow.stop()
fast.stop()
}
@Test
def canNotUseActorsOfDifferentTypesInSameDispatcher(): Unit = {
val first = actorOf[FirstActor]
val second = actorOf[SecondActor]
val first = actorOf(Props[FirstActor].withDispatcher(sharedActorDispatcher))
first.start()
intercept[IllegalActorStateException] {
second.start()
actorOf(Props[SecondActor].withDispatcher(sharedActorDispatcher))
}
first.stop()
}
@Test
def canNotUseActorsOfDifferentSubTypesInSameDispatcher(): Unit = {
val parent = actorOf[ParentActor]
val child = actorOf[ChildActor]
val parent = actorOf(Props[ParentActor].withDispatcher(parentActorDispatcher))
parent.start()
intercept[IllegalActorStateException] {
child.start()
val child = actorOf(Props[ChildActor].withDispatcher(parentActorDispatcher))
child.stop()
}
parent.stop()
}
}

View file

@ -3,20 +3,17 @@ package akka.actor.dispatch
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import org.scalatest.junit.JUnitSuite
import org.junit.Test
import akka.dispatch.{ Dispatchers, Dispatcher }
import akka.actor.Actor
import Actor._
import akka.actor.Actor._
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
import akka.testkit.{ filterEvents, EventFilter }
import akka.dispatch.{ PinnedDispatcher, Dispatchers, Dispatcher }
import akka.actor.{ Props, Actor }
object DispatcherActorSpec {
class TestActor extends Actor {
self.dispatcher = Dispatchers.newDispatcher(self.uuid.toString).build
def receive = {
case "Hello"
self.reply("World")
case "Failure"
throw new RuntimeException("Expected exception; to test fault-tolerance")
case "Hello" self.reply("World")
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
}
}
@ -24,7 +21,6 @@ object DispatcherActorSpec {
val oneWay = new CountDownLatch(1)
}
class OneWayTestActor extends Actor {
self.dispatcher = Dispatchers.newDispatcher(self.uuid.toString).build
def receive = {
case "OneWay" OneWayTestActor.oneWay.countDown()
}
@ -37,7 +33,7 @@ class DispatcherActorSpec extends JUnitSuite {
@Test
def shouldTell = {
val actor = actorOf[OneWayTestActor].start()
val actor = actorOf(Props[OneWayTestActor].withDispatcher(new PinnedDispatcher()))
val result = actor ! "OneWay"
assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
actor.stop()
@ -45,7 +41,7 @@ class DispatcherActorSpec extends JUnitSuite {
@Test
def shouldSendReplySync = {
val actor = actorOf[TestActor].start()
val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher()))
val result = (actor.?("Hello", 10000)).as[String]
assert("World" === result.get)
actor.stop()
@ -53,7 +49,7 @@ class DispatcherActorSpec extends JUnitSuite {
@Test
def shouldSendReplyAsync = {
val actor = actorOf[TestActor].start()
val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher()))
val result = (actor ? "Hello").as[String]
assert("World" === result.get)
actor.stop()
@ -62,7 +58,7 @@ class DispatcherActorSpec extends JUnitSuite {
@Test
def shouldSendReceiveException = {
filterEvents(EventFilter[RuntimeException]("Expected")) {
val actor = actorOf[TestActor].start()
val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher()))
try {
(actor ? "Failure").get
fail("Should have thrown an exception")
@ -85,19 +81,13 @@ class DispatcherActorSpec extends JUnitSuite {
val latch = new CountDownLatch(100)
val start = new CountDownLatch(1)
val fastOne = actorOf(
new Actor {
self.dispatcher = throughputDispatcher
def receive = { case "sabotage" works.set(false) }
}).start()
Props(self { case "sabotage" works.set(false) }).withDispatcher(throughputDispatcher))
val slowOne = actorOf(
new Actor {
self.dispatcher = throughputDispatcher
def receive = {
case "hogexecutor" start.await
case "ping" if (works.get) latch.countDown()
}
}).start()
Props(self {
case "hogexecutor" start.await
case "ping" if (works.get) latch.countDown()
}).withDispatcher(throughputDispatcher))
slowOne ! "hogexecutor"
(1 to 100) foreach { _ slowOne ! "ping" }
@ -122,19 +112,15 @@ class DispatcherActorSpec extends JUnitSuite {
val ready = new CountDownLatch(1)
val fastOne = actorOf(
new Actor {
self.dispatcher = throughputDispatcher
def receive = { case "ping" if (works.get) latch.countDown(); self.stop() }
}).start()
Props(self {
case "ping" if (works.get) latch.countDown(); self.stop()
}).withDispatcher(throughputDispatcher))
val slowOne = actorOf(
new Actor {
self.dispatcher = throughputDispatcher
def receive = {
case "hogexecutor" ready.countDown(); start.await
case "ping" works.set(false); self.stop()
}
}).start()
Props(self {
case "hogexecutor" ready.countDown(); start.await
case "ping" works.set(false); self.stop()
}).withDispatcher(throughputDispatcher))
slowOne ! "hogexecutor"
slowOne ! "ping"

View file

@ -289,6 +289,20 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd
}
}
"firstCompletedOf" in {
val futures = Vector.fill[Future[Int]](10)(new DefaultPromise[Int]()) :+ new KeptPromise[Int](Right(5))
Futures.firstCompletedOf(futures).get must be(5)
}
"find" in {
val futures = for (i 1 to 10) yield Future { i }
val result = Futures.find[Int](_ == 3)(futures)
result.get must be(Some(3))
val notFound = Futures.find[Int](_ == 11)(futures)
notFound.get must be(None)
}
"fold" in {
val actors = (1 to 10).toList map { _
actorOf(new Actor {

View file

@ -4,12 +4,12 @@ import org.scalatest.matchers.MustMatchers
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import akka.actor.{ Actor, ActorRegistry, NullChannel }
import akka.actor.Actor.{ actorOf }
import java.util.concurrent.{ TimeUnit, CountDownLatch, BlockingQueue }
import java.util.{ Queue }
import akka.util._
import akka.util.Duration._
import akka.actor.{ LocalActorRef, Actor, ActorRegistry, NullChannel }
@RunWith(classOf[JUnitRunner])
abstract class MailboxSpec extends WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach {
@ -84,7 +84,7 @@ abstract class MailboxSpec extends WordSpec with MustMatchers with BeforeAndAfte
new MessageInvocation(
actorOf(new Actor { //Dummy actor
def receive = { case _ }
}), msg, NullChannel)
}).asInstanceOf[LocalActorRef], msg, NullChannel)
}
def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) {

View file

@ -4,23 +4,19 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit }
import org.scalatest.junit.JUnitSuite
import org.junit.{ Test, Before, After }
import akka.dispatch.Dispatchers
import akka.actor.Actor
import Actor._
import akka.actor.Actor._
import akka.event.EventHandler
import akka.testkit.TestEvent._
import akka.testkit.EventFilter
import akka.dispatch.{ PinnedDispatcher, Dispatchers }
import akka.actor.{ Props, Actor }
object PinnedActorSpec {
class TestActor extends Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
def receive = {
case "Hello"
self.reply("World")
case "Failure"
throw new RuntimeException("Expected exception; to test fault-tolerance")
case "Hello" self.reply("World")
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
}
}
}
@ -43,12 +39,7 @@ class PinnedActorSpec extends JUnitSuite {
@Test
def shouldTell {
var oneWay = new CountDownLatch(1)
val actor = actorOf(new Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
def receive = {
case "OneWay" oneWay.countDown()
}
}).start()
val actor = actorOf(Props(self { case "OneWay" oneWay.countDown() }).withDispatcher(new PinnedDispatcher()))
val result = actor ! "OneWay"
assert(oneWay.await(1, TimeUnit.SECONDS))
actor.stop()
@ -56,7 +47,7 @@ class PinnedActorSpec extends JUnitSuite {
@Test
def shouldSendReplySync = {
val actor = actorOf[TestActor].start()
val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher()))
val result = (actor.?("Hello", 10000)).as[String]
assert("World" === result.get)
actor.stop()
@ -64,7 +55,7 @@ class PinnedActorSpec extends JUnitSuite {
@Test
def shouldSendReplyAsync = {
val actor = actorOf[TestActor].start()
val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher()))
val result = (actor ? "Hello").as[String]
assert("World" === result.get)
actor.stop()
@ -72,7 +63,7 @@ class PinnedActorSpec extends JUnitSuite {
@Test
def shouldSendReceiveException = {
val actor = actorOf[TestActor].start()
val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher()))
try {
(actor ? "Failure").get
fail("Should have thrown an exception")

View file

@ -1,10 +1,9 @@
package akka.dispatch
import akka.actor.Actor._
import akka.actor.Actor
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.CountDownLatch
import akka.actor.{ Props, LocalActorRef, Actor }
class PriorityDispatcherSpec extends WordSpec with MustMatchers {
@ -27,15 +26,14 @@ class PriorityDispatcherSpec extends WordSpec with MustMatchers {
throughput = 1,
mailboxType = mboxType)
val actor = actorOf(new Actor {
self.dispatcher = dispatcher
val actor = actorOf(Props(new Actor {
var acc: List[Int] = Nil
def receive = {
case i: Int acc = i :: acc
case 'Result self tryReply acc
}
}).start()
}).withDispatcher(dispatcher)).asInstanceOf[LocalActorRef]
dispatcher.suspend(actor) //Make sure the actor isn't treating any messages, let it buffer the incoming messages

View file

@ -63,11 +63,10 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldFindThingsFromLocalActorRegistry {
Actor.registry.local.shutdownAll
val actor = actorOf[TestActor]("test-actor-1")
actor.start
val found = Actor.registry.local.find({ case a: ActorRef if a.actor.isInstanceOf[TestActor] a })
val actor = actorOf[TestActor]("test-actor-1").start()
val found: Option[LocalActorRef] = Actor.registry.local.find({ case a: LocalActorRef if a.actorInstance.get().isInstanceOf[TestActor] a })
assert(found.isDefined)
assert(found.get.actor.isInstanceOf[TestActor])
assert(found.get.actorInstance.get().isInstanceOf[TestActor])
assert(found.get.address === "test-actor-1")
actor.stop
}
@ -75,13 +74,13 @@ 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").start()
val actor2 = actorOf[TestActor]("test-actor-2").start()
val actors = Actor.registry.local.actors
assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor])
assert(actors.head.address === "test-actor-2")
assert(actors.last.actor.isInstanceOf[TestActor])
assert(actors.last.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor])
assert(actors.last.address === "test-actor-1")
actor1.stop
actor2.stop

View file

@ -119,8 +119,6 @@ class SchedulerSpec extends JUnitSuite {
val pingLatch = new CountDownLatch(6)
val actor = actorOf(new Actor {
self.lifeCycle = Permanent
def receive = {
case Ping pingLatch.countDown()
case Crash throw new Exception("CRASH")

View file

@ -6,12 +6,10 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import akka.performance.trading.domain._
import akka.performance.trading.common._
import akka.actor.ActorRef
import akka.actor.Actor
import akka.actor.Actor.actorOf
import akka.dispatch.Dispatchers
import akka.actor.PoisonPill
import akka.event.EventHandler
import akka.actor.{ Props, ActorRef, Actor, PoisonPill }
abstract class AkkaPerformanceTest extends BenchmarkScenarios {
@ -38,7 +36,7 @@ abstract class AkkaPerformanceTest extends BenchmarkScenarios {
val receivers = tradingSystem.orderReceivers.toIndexedSeq
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
actorOf(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs))
actorOf(Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs)).withDispatcher(clientDispatcher))
}).toList
clients.foreach(_.start)
@ -54,9 +52,6 @@ abstract class AkkaPerformanceTest extends BenchmarkScenarios {
}
class Client(orderReceiver: ActorRef, orders: List[Order], latch: CountDownLatch, repeat: Int, delayMs: Int) extends Actor {
self.dispatcher = clientDispatcher
def this(orderReceiver: ActorRef, orders: List[Order], latch: CountDownLatch, repeat: Int) {
this(orderReceiver, orders, latch, repeat, 0)
}

View file

@ -16,13 +16,9 @@ trait MatchingEngine {
}
class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook], disp: Option[MessageDispatcher])
class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
extends Actor with MatchingEngine {
for (d disp) {
self.dispatcher = d
}
var standby: Option[ActorRef] = None
def receive = {

View file

@ -25,14 +25,9 @@ trait OrderReceiver {
}
class AkkaOrderReceiver(disp: Option[MessageDispatcher])
extends Actor with OrderReceiver {
class AkkaOrderReceiver extends Actor with OrderReceiver {
type ME = ActorRef
for (d disp) {
self.dispatcher = d
}
def receive = {
case routing @ MatchingEngineRouting(mapping)
refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]])

View file

@ -3,9 +3,8 @@ package akka.performance.trading.common
import akka.performance.trading.domain.Orderbook
import akka.performance.trading.domain.OrderbookRepository
import akka.actor.Actor._
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.dispatch.MessageDispatcher
import akka.actor.{ Props, ActorRef, PoisonPill }
trait TradingSystem {
type ME
@ -69,7 +68,10 @@ class AkkaTradingSystem extends TradingSystem {
}
def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) =
actorOf(new AkkaMatchingEngine(meId, orderbooks, meDispatcher))
meDispatcher match {
case Some(d) actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d))
case _ actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)))
}
override def createOrderReceivers: List[ActorRef] = {
(1 to 10).toList map (i createOrderReceiver())
@ -87,8 +89,10 @@ class AkkaTradingSystem extends TradingSystem {
MatchingEngineRouting(Map() ++ rules)
}
def createOrderReceiver() =
actorOf(new AkkaOrderReceiver(orDispatcher))
def createOrderReceiver() = orDispatcher match {
case Some(d) actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d))
case _ actorOf(Props(new AkkaOrderReceiver()))
}
override def start() {
for (MatchingEngineInfo(p, s, o) matchingEngines) {

View file

@ -7,8 +7,7 @@ import akka.performance.trading.domain.Order
import akka.performance.trading.domain.Orderbook
import akka.performance.trading.common.AkkaMatchingEngine
class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook], disp: Option[MessageDispatcher])
extends AkkaMatchingEngine(meId, orderbooks, disp) {
class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook]) extends AkkaMatchingEngine(meId, orderbooks) {
override def handleOrder(order: Order) {
orderbooksMap.get(order.orderbookSymbol) match {

View file

@ -6,8 +6,7 @@ import akka.event.EventHandler
import akka.performance.trading.domain._
import akka.performance.trading.common.AkkaOrderReceiver
class OneWayOrderReceiver(disp: Option[MessageDispatcher])
extends AkkaOrderReceiver(disp) {
class OneWayOrderReceiver extends AkkaOrderReceiver {
override def placeOrder(order: Order) = {
val matchingEngine = matchingEngineForOrderbook.get(order.orderbookSymbol)

View file

@ -5,16 +5,18 @@ import java.util.concurrent.TimeUnit
import org.junit.Test
import akka.actor.Actor.actorOf
import akka.actor.ActorRef
import akka.performance.trading.common.AkkaPerformanceTest
import akka.performance.trading.common.Rsp
import akka.performance.trading.domain._
import akka.actor.{ Props, ActorRef }
class OneWayPerformanceTest extends AkkaPerformanceTest {
override def createTradingSystem: TS = new OneWayTradingSystem {
override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) =
actorOf(new OneWayMatchingEngine(meId, orderbooks, meDispatcher) with LatchMessageCountDown)
override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match {
case Some(d) actorOf(Props(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown).withDispatcher(d))
case _ actorOf(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown)
}
}
override def placeOrder(orderReceiver: ActorRef, order: Order): Rsp = {

View file

@ -1,16 +1,20 @@
package akka.performance.trading.oneway
import akka.actor.Actor.actorOf
import akka.actor.ActorRef
import akka.performance.trading.common.AkkaTradingSystem
import akka.performance.trading.domain.Orderbook
import akka.actor.{ Props, ActorRef }
class OneWayTradingSystem extends AkkaTradingSystem {
override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) =
actorOf(new OneWayMatchingEngine(meId, orderbooks, meDispatcher))
override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match {
case Some(d) actorOf(Props(new OneWayMatchingEngine(meId, orderbooks)).withDispatcher(d))
case _ actorOf(Props(new OneWayMatchingEngine(meId, orderbooks)))
}
override def createOrderReceiver() =
actorOf(new OneWayOrderReceiver(orDispatcher))
override def createOrderReceiver() = orDispatcher match {
case Some(d) actorOf(Props[OneWayOrderReceiver].withDispatcher(d))
case _ actorOf(Props[OneWayOrderReceiver])
}
}

View file

@ -6,9 +6,10 @@ import akka.dispatch.{ KeptPromise, Future }
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.Actor._
import akka.testkit.Testing._
import akka.actor.{ TypedActor, Actor }
import akka.actor.{ TypedActor, Actor, Props }
import akka.testkit.{ TestLatch, filterEvents, EventFilter, filterException }
import akka.util.duration._
import akka.config.Supervision.OneForOneStrategy
object ActorPoolSpec {
@ -22,6 +23,8 @@ object ActorPoolSpec {
new KeptPromise(Right(x * x))
}
}
val faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000)
}
class ActorPoolSpec extends WordSpec with MustMatchers {
@ -34,7 +37,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
val count = new AtomicInteger(0)
val pool = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with FixedCapacityStrategy with SmallestMailboxSelector {
Props(new Actor with DefaultActorPool with FixedCapacityStrategy with SmallestMailboxSelector {
def factory = actorOf(new Actor {
def receive = {
case _
@ -49,7 +52,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def partialFill = true
def instance = factory
def receive = _route
}).start()
}).withFaultHandler(faultHandler))
val successes = TestLatch(2)
val successCounter = actorOf(new Actor {
@ -74,7 +77,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
"pass ticket #705" in {
val pool = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicFilter {
def lowerBound = 2
def upperBound = 20
def rampupRate = 0.1
@ -93,7 +96,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
}
}
})
}).start()
}).withFaultHandler(faultHandler))
try {
(for (count 1 to 500) yield pool.?("Test", 20000)) foreach {
@ -112,7 +115,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
val count = new AtomicInteger(0)
val pool = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
def factory = actorOf(new Actor {
def receive = {
case n: Int
@ -129,7 +132,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def selectionCount = 1
def instance = factory
def receive = _route
}).start()
}).withFaultHandler(faultHandler))
// first message should create the minimum number of delgates
@ -177,7 +180,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
val count = new AtomicInteger(0)
val pool = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
def factory = actorOf(new Actor {
def receive = {
case n: Int
@ -195,7 +198,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def selectionCount = 1
def instance = factory
def receive = _route
}).start()
}).withFaultHandler(faultHandler))
var loops = 0
def loop(t: Int) = {
@ -231,7 +234,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
val delegates = new java.util.concurrent.ConcurrentHashMap[String, String]
val pool1 = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter {
Props(new Actor with DefaultActorPool with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter {
def factory = actorOf(new Actor {
def receive = {
case _
@ -246,7 +249,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def partialFill = true
def instance = factory
def receive = _route
}).start()
}).withFaultHandler(faultHandler))
pool1 ! "a"
pool1 ! "b"
@ -260,7 +263,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
delegates.clear()
val pool2 = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter {
Props(new Actor with DefaultActorPool with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter {
def factory = actorOf(new Actor {
def receive = {
case _
@ -275,7 +278,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def partialFill = false
def instance = factory
def receive = _route
}).start()
}).withFaultHandler(faultHandler))
pool2 ! "a"
pool2 ! "b"
@ -290,7 +293,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
val latch = TestLatch(10)
val pool = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
def factory = actorOf(new Actor {
def receive = {
case n: Int
@ -309,7 +312,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def backoffThreshold = 0.50
def instance = factory
def receive = _route
}).start()
}).withFaultHandler(faultHandler))
// put some pressure on the pool
@ -336,7 +339,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
"support typed actors" in {
import RoutingSpec._
import TypedActor._
def createPool = new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
def createPool = new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
def lowerBound = 1
def upperBound = 5
def pressureThreshold = 1
@ -349,7 +352,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def receive = _route
}
val pool = createProxy[Foo](createPool)
val pool = createProxy[Foo](createPool, Props().withFaultHandler(faultHandler))
val results = for (i 1 to 100) yield (i, pool.sq(i, 100))
@ -364,7 +367,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
var keepDying = false
val pool1 = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter {
def lowerBound = 2
def upperBound = 5
def rampupRate = 0.1
@ -385,10 +388,10 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
case _ pingCount.incrementAndGet
}
}).start()
}).start()
}).withFaultHandler(faultHandler))
val pool2 = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter {
def lowerBound = 2
def upperBound = 5
def rampupRate = 0.1
@ -400,7 +403,6 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def receive = _route
def pressureThreshold = 1
def factory = actorOf(new Actor {
self.lifeCycle = Permanent
if (deathCount.get > 5) deathCount.set(0)
if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") }
def receive = {
@ -410,10 +412,10 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
case _ pingCount.incrementAndGet
}
}).start()
}).start()
}).withFaultHandler(faultHandler))
val pool3 = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with RoundRobinSelector with BasicFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with RoundRobinSelector with BasicFilter {
def lowerBound = 2
def upperBound = 5
def rampupRate = 0.1
@ -424,8 +426,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
def instance = factory
def receive = _route
def pressureThreshold = 1
def factory = actorOf(new Actor {
self.lifeCycle = Temporary
def factory = actorOf(Props(new Actor {
if (deathCount.get > 5) deathCount.set(0)
if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") }
def receive = {
@ -434,8 +435,8 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
throw new RuntimeException
case _ pingCount.incrementAndGet
}
}).start()
}).start()
}).withLifeCycle(Temporary))
}).withFaultHandler(faultHandler))
// default lifecycle
// actor comes back right away
@ -508,14 +509,10 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
val deathCount = new AtomicInteger(0)
var keepDying = false
trait LimitedTrapSupervisionConfig extends ActorPoolSupervisionConfig {
def poolFaultHandler = OneForOneStrategy(List(classOf[IllegalStateException]), 5, 1000)
}
object BadState
val pool1 = actorOf(
new Actor with DefaultActorPool with LimitedTrapSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter {
def lowerBound = 2
def upperBound = 5
def rampupRate = 0.1
@ -538,7 +535,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers {
case _ pingCount.incrementAndGet
}
}).start()
}).start()
}).withFaultHandler(OneForOneStrategy(List(classOf[IllegalStateException]), 5, 1000)))
// actor comes back right away
pingCount.set(0)

View file

@ -3,7 +3,6 @@ package akka.routing
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.routing._
import akka.routing.Router
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.Actor._
import akka.actor.{ ActorRef, Actor }

View file

@ -18,25 +18,30 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
val shutdownLatch = new CountDownLatch(1)
val actor = Routing.actorOf("foo", List(newActor(0, Some(shutdownLatch)),
newActor(1, Some(shutdownLatch))),
new ScatterGatherFirstCompletedRouter()).start()
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf(props)
actor ! Broadcast(Stop(Some(0)))
shutdownLatch.await(5, TimeUnit.SECONDS) must be(true)
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"throw an exception, if all the connections have stopped" in {
val shutdownLatch = new CountDownLatch(2)
val actor = Routing.actorOf("foo", List(newActor(0, Some(shutdownLatch)),
newActor(1, Some(shutdownLatch))),
new ScatterGatherFirstCompletedRouter()).start()
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf(props)
actor ! Broadcast(Stop())
@ -50,35 +55,48 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
"return the first response from connections, when all of them replied" in {
val actor = Routing.actorOf("foo", List(newActor(0), newActor(1)),
new ScatterGatherFirstCompletedRouter()).start()
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List(newActor(0), newActor(1)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf(props)
(actor ? Broadcast("Hi!")).get.asInstanceOf[Int] must be(0)
}
"return the first response from connections, when some of them failed to reply" in {
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List(newActor(0), newActor(1)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf("foo", List(newActor(0), newActor(1)),
new ScatterGatherFirstCompletedRouter()).start()
val actor = Routing.actorOf(props)
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"be started when constructed" in {
val actor = Routing.actorOf("foo", List(newActor(0)),
new ScatterGatherFirstCompletedRouter()).start()
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List(newActor(0)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf(props)
actor.isRunning must be(true)
}
"throw IllegalArgumentException at construction when no connections" in {
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List())
.withRouter(() new ScatterGatherFirstCompletedRouter())
try {
Routing.actorOf("foo", List(),
new ScatterGatherFirstCompletedRouter()).start()
Routing.actorOf(props)
fail()
} catch {
case e: IllegalArgumentException
@ -104,7 +122,12 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
connections = connections :+ connection
}
val actor = Routing.actorOf("foo", connections, new ScatterGatherFirstCompletedRouter()).start()
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(connections)
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf(props)
for (i 0 until iterationCount) {
for (k 0 until connectionCount) {
@ -141,7 +164,12 @@ class Ticket1111Spec extends WordSpec with MustMatchers {
}
}).start()
val actor = Routing.actorOf("foo", List(connection1, connection2), new ScatterGatherFirstCompletedRouter()).start()
val props = RoutedProps.apply()
.withDeployId("foo")
.withConnections(List(connection1, connection2))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = Routing.actorOf(props)
actor ! Broadcast(1)
actor ! Broadcast("end")

View file

@ -5,13 +5,14 @@ import akka.actor._
import akka.routing._
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.config.Supervision.OneForOneStrategy
class Ticket703Spec extends WordSpec with MustMatchers {
"A ? call to an actor pool" should {
"reuse the proper timeout" in {
val actorPool = actorOf(
new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
def lowerBound = 2
def upperBound = 20
def rampupRate = 0.1
@ -27,7 +28,7 @@ class Ticket703Spec extends WordSpec with MustMatchers {
self.tryReply("Response")
}
})
}).start()
}).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, 1000)))
(actorPool.?("Ping", 7000)).await.result must be === Some("Response")
}
}

View file

@ -0,0 +1,15 @@
package akka.routing;
/**
* A Factory responsible for creating {@link Router} instances. It makes Java compatability possible for users that
* want to provide their own router instance.
*/
public interface RouterFactory {
/**
* Creates a new Router instance.
*
* @return the newly created Router instance.
*/
Router newRouter();
}

View file

@ -30,7 +30,7 @@ class AkkaException(message: String = "", cause: Throwable = null) extends Runti
def stackTraceToString = {
val trace = getStackTrace
val sb = new StringBuffer
val sb = new StringBuilder
for (i 0 until trace.length)
sb.append("\tat %s\n" format trace(i))
sb.toString

View file

@ -36,7 +36,7 @@ sealed trait LifeCycleMessage extends Serializable
*/
sealed trait AutoReceivedMessage { self: LifeCycleMessage }
case class HotSwap(code: ActorRef Actor.Receive, discardOld: Boolean = true) extends AutoReceivedMessage with LifeCycleMessage {
case class HotSwap(code: SelfActorRef Actor.Receive, discardOld: Boolean = true) extends AutoReceivedMessage with LifeCycleMessage {
/**
* Java API
@ -137,6 +137,7 @@ object Timeout {
/**
* The default timeout, based on the config setting 'akka.actor.timeout'
*/
@BeanProperty
implicit val default = new Timeout(Actor.TIMEOUT)
/**
@ -172,8 +173,8 @@ object Actor {
*/
type Receive = PartialFunction[Any, Unit]
private[actor] val actorRefInCreation = new ThreadLocal[Stack[ActorRef]] {
override def initialValue = Stack[ActorRef]()
private[actor] val actorRefInCreation = new ThreadLocal[Stack[ScalaActorRef with SelfActorRef]] {
override def initialValue = Stack[ScalaActorRef with SelfActorRef]()
}
private[akka] val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis
@ -343,7 +344,7 @@ object Actor {
* </pre>
*/
def actorOf[T <: Actor](creator: T, address: String): ActorRef = {
createActor(address, () new LocalActorRef(() creator, address))
createActor(address, () new LocalActorRef(Props(creator = () creator), address))
}
/**
@ -366,28 +367,22 @@ object Actor {
* JAVA API
*/
def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = {
createActor(address, () new LocalActorRef(() creator.create, address))
createActor(address, () new LocalActorRef(Props(creator = () creator.create), address))
}
def actorOf(props: Props): ActorRef = actorOf(props, newUuid.toString)
//TODO FIXME
def actorOf(props: Props): ActorRef = {
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?
/*val address = props.deployId match { //TODO handle deployId separately from address?
case "" | null newUuid().toString
case other other
}
val newActor = new LocalActorRef(props.creator, address)
newActor.dispatcher = props.dispatcher
newActor.faultHandler = props.faultHandler
newActor.lifeCycle = props.lifeCycle
newActor.timeout = props.timeout.duration.toMillis
newActor.receiveTimeout = props.receiveTimeout.map(_.toMillis)
props.supervisor.foreach(newActor.link(_))
newActor.start
} */
new LocalActorRef(props, address).start()
}
def localActorOf[T <: Actor: Manifest]: ActorRef = {
@ -407,11 +402,11 @@ object Actor {
}
def localActorOf[T <: Actor](factory: T): ActorRef = {
new LocalActorRef(() factory, newUuid().toString)
new LocalActorRef(Props(creator = () factory), newUuid().toString)
}
def localActorOf[T <: Actor](factory: T, address: String): ActorRef = {
new LocalActorRef(() factory, address)
new LocalActorRef(Props(creator = () factory), address)
}
/**
@ -432,13 +427,9 @@ object Actor {
* </pre>
*/
def spawn(body: Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = {
case object Spawn
actorOf(new Actor() {
self.dispatcher = dispatcher
def receive = {
case Spawn try { body } finally { self.stop() }
}
}).start() ! Spawn
actorOf(Props(new Actor() {
def receive = { case "go" try { body } finally { self.stop() } }
}).withDispatcher(dispatcher)) ! "go"
}
/**
@ -464,7 +455,7 @@ object Actor {
}
private[akka] def newLocalActorRef(clazz: Class[_ <: Actor], address: String): ActorRef = {
new LocalActorRef(() {
new LocalActorRef(Props(creator = () {
import ReflectiveAccess.{ createInstance, noParams, noArgs }
createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs) match {
case Right(actor) actor
@ -480,7 +471,7 @@ object Actor {
"\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)
}), address)
}
private[akka] def newClusterActorRef(factory: () ActorRef, address: String, deploy: Deploy): ActorRef =
@ -548,7 +539,7 @@ object Actor {
* <p/>
* Here you find functions like:
* - !, ? and forward
* - link, unlink, startLink etc
* - link, unlink etc
* - start, stop
* - etc.
*
@ -593,7 +584,7 @@ trait Actor {
* the 'forward' function.
*/
@transient
val someSelf: Some[ActorRef] = {
val someSelf: Some[ScalaActorRef with SelfActorRef] = {
val refStack = Actor.actorRefInCreation.get
if (refStack.isEmpty) throw new ActorInitializationException(
"\n\tYou can not create an instance of an " + getClass.getName + " explicitly using 'new MyActor'." +
@ -619,7 +610,7 @@ trait Actor {
* Mainly for internal use, functions as the implicit sender references when invoking
* one of the message send functions ('!' and '?').
*/
def optionSelf: Option[ActorRef] = someSelf
def optionSelf: Option[ScalaActorRef with SelfActorRef] = someSelf
/**
* The 'self' field holds the ActorRef for this actor.
@ -649,7 +640,7 @@ trait Actor {
* </pre>
*/
@transient
implicit val self: ScalaActorRef = someSelf.get
implicit val self = someSelf.get
/**
* User overridable callback/setting.

File diff suppressed because it is too large Load diff

View file

@ -155,7 +155,7 @@ trait ForwardableChannel extends UntypedChannel with AvailableChannel[Any] {
}
object ForwardableChannel {
implicit def someS2FC(sender: Some[ActorRef]): ForwardableChannel = sender.get
implicit def someIS2FC(implicit sender: Some[ActorRef]): ForwardableChannel = sender.get
implicit def someS2FC(sender: Some[SelfActorRef]): ForwardableChannel = sender.get
implicit def someIS2FC(implicit sender: Some[SelfActorRef]): ForwardableChannel = sender.get
}

View file

@ -193,16 +193,16 @@ object Deployer extends ActorDeployer {
// akka.actor.deployment.<address>.clustered.replicas
// --------------------------------
val replicationFactor = {
if (router == Direct) ReplicationFactor(1)
if (router == Direct) new ReplicationFactor(1)
else {
clusteredConfig.getAny("replication-factor", "0") match {
case "auto" AutoReplicationFactor
case "0" ZeroReplicationFactor
case nrOfReplicas: String
try {
ReplicationFactor(nrOfReplicas.toInt)
new ReplicationFactor(nrOfReplicas.toInt)
} catch {
case e: NumberFormatException
case e: Exception
throw new ConfigurationException(
"Config option [" + addressPath +
".clustered.replicas] needs to be either [\"auto\"] or [0-N] - was [" +

View file

@ -80,17 +80,18 @@ object DeploymentConfig {
// --------------------------------
// --- Replicas
// --------------------------------
sealed case class ReplicationFactor(val factor: Int) {
case class ReplicationFactor(val factor: Int) {
if (factor < 0) throw new IllegalArgumentException("replication-factor can not be negative")
}
// For Java API
case class AutoReplicationFactor() extends ReplicationFactor(-1)
case class ZeroReplicationFactor() extends ReplicationFactor(0)
class AutoReplicationFactor extends ReplicationFactor(-1)
class ZeroReplicationFactor extends ReplicationFactor(0)
// For Scala API
case object AutoReplicationFactor extends ReplicationFactor(-1)
case object ZeroReplicationFactor extends ReplicationFactor(0)
case object AutoReplicationFactor extends AutoReplicationFactor
case object ZeroReplicationFactor extends ZeroReplicationFactor
// --------------------------------
// --- Replication
@ -125,12 +126,12 @@ object DeploymentConfig {
sealed trait ReplicationStrategy
// For Java API
case class WriteBehind() extends ReplicationStrategy
case class WriteThrough() extends ReplicationStrategy
sealed class WriteBehind extends ReplicationStrategy
sealed class WriteThrough extends ReplicationStrategy
// For Scala API
case object WriteBehind extends ReplicationStrategy
case object WriteThrough extends ReplicationStrategy
case object WriteBehind extends WriteBehind
case object WriteThrough extends WriteThrough
// --------------------------------
// --- Helper methods for parsing

View file

@ -200,7 +200,7 @@ trait IO {
if (!reinvoked && (_next eq Idle) && _messages.nonEmpty) {
try {
reinvoked = true
while ((_next eq Idle) && _messages.nonEmpty) self invoke _messages.dequeue
while ((_next eq Idle) && _messages.nonEmpty) self.asInstanceOf[LocalActorRef] invoke _messages.dequeue
} finally {
reinvoked = false
}

View file

@ -8,9 +8,9 @@ import akka.AkkaException
import akka.util._
import ReflectiveAccess._
import Actor._
import java.util.concurrent.{ CopyOnWriteArrayList, ConcurrentHashMap }
import java.util.concurrent.{ CopyOnWriteArrayList }
import akka.config.Supervision._
import collection.mutable.ListBuffer
class SupervisorException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null);
@ -110,10 +110,10 @@ case class SupervisorFactory(val config: SupervisorConfig) {
sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (ActorRef, MaximumNumberOfRestartsWithinTimeRangeReached) Unit) {
import Supervisor._
private val _childActors = new ConcurrentHashMap[String, List[ActorRef]]
private val _childActors = new CopyOnWriteArrayList[ActorRef]
private val _childSupervisors = new CopyOnWriteArrayList[Supervisor]
private[akka] val supervisor = localActorOf(new SupervisorActor(handler, maxRestartsHandler)).start()
private[akka] val supervisor = actorOf(Props(new SupervisorActor(maxRestartsHandler)).withFaultHandler(handler))
def uuid = supervisor.uuid
@ -125,39 +125,39 @@ sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (Act
def link(child: ActorRef) = supervisor.link(child)
def startLink(child: ActorRef) = supervisor.startLink(child)
def unlink(child: ActorRef) = supervisor.unlink(child)
def children: List[ActorRef] =
_childActors.values.toArray.toList.asInstanceOf[List[List[ActorRef]]].flatten
def children: List[ActorRef] = {
val buf = new ListBuffer[ActorRef]
val i = _childActors.iterator()
while (i.hasNext) buf += i.next()
buf.toList
}
def childSupervisors: List[Supervisor] =
_childActors.values.toArray.toList.asInstanceOf[List[Supervisor]]
def childSupervisors: List[Supervisor] = {
val buf = new ListBuffer[Supervisor]
val i = _childSupervisors.iterator()
while (i.hasNext) buf += i.next()
buf.toList
}
def configure(config: SupervisorConfig): Unit = config match {
case SupervisorConfig(_, servers, _)
servers foreach {
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()
servers.map(server
server match {
case Supervise(actorRef, lifeCycle, registerAsRemoteService)
actorRef.start()
val className = actorRef.actor.getClass.getName
val currentActors = {
val list = _childActors.get(className)
if (list eq null) List[ActorRef]()
else list
}
_childActors.put(className, actorRef :: currentActors)
actorRef.lifeCycle = lifeCycle
supervisor.link(actorRef)
if (ClusterModule.isEnabled && registerAsRemoteService)
Actor.remote.register(actorRef)
case supervisorConfig @ SupervisorConfig(_, _, _) // recursive supervisor configuration
val childSupervisor = Supervisor(supervisorConfig)
supervisor.link(childSupervisor.supervisor)
_childSupervisors.add(childSupervisor)
})
_childActors.add(actorRef) //TODO Why do we keep this here, mem leak?
if (ClusterModule.isEnabled && registerAsRemoteService)
Actor.remote.register(actorRef)
case supervisorConfig @ SupervisorConfig(_, _, _) // recursive supervisor configuration
val childSupervisor = Supervisor(supervisorConfig)
supervisor.link(childSupervisor.supervisor)
_childSupervisors.add(childSupervisor)
}
}
}
@ -166,8 +166,7 @@ sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (Act
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
final class SupervisorActor private[akka] (handler: FaultHandlingStrategy, maxRestartsHandler: (ActorRef, MaximumNumberOfRestartsWithinTimeRangeReached) Unit) extends Actor {
self.faultHandler = handler
final class SupervisorActor private[akka] (maxRestartsHandler: (ActorRef, MaximumNumberOfRestartsWithinTimeRangeReached) Unit) extends Actor {
override def postStop(): Unit = {
val i = self.linkedActors.values.iterator

View file

@ -212,7 +212,7 @@ object TypedActor {
private[akka] def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: (AtomVar[R]) Actor, props: Props, loader: ClassLoader): R = {
val proxyVar = new AtomVar[R]
configureAndProxyLocalActorRef[R](interfaces, proxyVar, props.withCreator(() constructor(proxyVar)), loader)
configureAndProxyLocalActorRef[R](interfaces, proxyVar, props.withCreator(constructor(proxyVar)), loader)
}
private[akka] def createProxyAndTypedActor[R <: AnyRef, T <: R](interface: Class[_], constructor: T, props: Props, loader: ClassLoader): R =
@ -221,7 +221,7 @@ object TypedActor {
private[akka] def configureAndProxyLocalActorRef[T <: AnyRef](interfaces: Array[Class[_]], proxyVar: AtomVar[T], props: Props, loader: ClassLoader): T = {
//Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
val actorVar = new AtomVar[ActorRef](null)
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)).asInstanceOf[T]
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(props.timeout)).asInstanceOf[T]
proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive
val ref = actorOf(props)
actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet
@ -252,7 +252,7 @@ object TypedActor {
}
}
private[akka] class TypedActorInvocationHandler(actorVar: AtomVar[ActorRef]) extends InvocationHandler {
private[akka] class TypedActorInvocationHandler(actorVar: AtomVar[ActorRef])(implicit timeout: Timeout) extends InvocationHandler {
def actor = actorVar.get
def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = method.getName match {
@ -260,7 +260,6 @@ object TypedActor {
case "equals" (args.length == 1 && (proxy eq args(0)) || actor == getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean
case "hashCode" actor.hashCode.asInstanceOf[AnyRef]
case _
implicit val timeout = Timeout(actor.timeout)
MethodCall(method, args) match {
case m if m.isOneWay actor ! m; null //Null return value
case m if m.returnsFuture_? actor ? m

View file

@ -65,12 +65,12 @@ abstract class UntypedActor extends Actor {
/**
* Returns the 'self' reference with the API.
*/
def getContext(): ActorRef = self
def getContext(): SelfActorRef = self
/**
* Returns the 'self' reference with the API.
*/
def context(): ActorRef = self
def context(): SelfActorRef = self
/**
* Java API for become

View file

@ -5,11 +5,8 @@
package akka
package object actor {
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef =
ref.asInstanceOf[ScalaActorRef]
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef =
ref.asInstanceOf[ActorRef]
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
type Uuid = com.eaio.uuid.UUID

View file

@ -18,6 +18,11 @@ import com.eaio.uuid.UUID
import java.net.InetSocketAddress
import java.util.concurrent.{ ConcurrentSkipListSet }
import akka.cluster.metrics._
import akka.util.Duration
import akka.util.duration._
class ClusterException(message: String) extends AkkaException(message)
object ChangeListener {
@ -109,6 +114,76 @@ object NodeAddress {
}
}
/*
* Allows user to access metrics of a different nodes in the cluster. Changing metrics can be monitored
* using {@link MetricsAlterationMonitor}
* Metrics of the cluster nodes are distributed through ZooKeeper. For better performance, metrics are
* cached internally, and refreshed from ZooKeeper after an interval
*/
trait NodeMetricsManager {
/*
* Gets metrics of a local node directly from JMX monitoring beans/Hyperic Sigar
*/
def getLocalMetrics: NodeMetrics
/*
* Gets metrics of a specified node
* @param nodeName metrics of the node specified by the name will be returned
* @param useCached if <code>true</code>, returns metrics cached in the metrics manager,
* gets metrics directly from ZooKeeper otherwise
*/
def getMetrics(nodeName: String, useCached: Boolean = true): Option[NodeMetrics]
/*
* Gets cached metrics of all nodes in the cluster
*/
def getAllMetrics: Array[NodeMetrics]
/*
* Adds monitor that reacts, when specific conditions are satisfied
*/
def addMonitor(monitor: MetricsAlterationMonitor): Unit
/*
* Removes monitor
*/
def removeMonitor(monitor: MetricsAlterationMonitor): Unit
/*
* Removes metrics of s specified node from ZooKeeper and metrics manager cache
*/
def removeNodeMetrics(nodeName: String): Unit
/*
* Sets timeout after which metrics, cached in the metrics manager, will be refreshed from ZooKeeper
*/
def refreshTimeout_=(newValue: Duration): Unit
/*
* Timeout after which metrics, cached in the metrics manager, will be refreshed from ZooKeeper
*/
def refreshTimeout: Duration
/*
* Starts metrics manager. When metrics manager is started, it refreshes cache from ZooKeeper
* after <code>refreshTimeout</code>, and invokes plugged monitors
*/
def start(): NodeMetricsManager
/*
* Stops metrics manager. Stopped metrics manager doesn't refresh cache from ZooKeeper,
* and doesn't invoke plugged monitors
*/
def stop(): Unit
/*
* If the value is <code>true</code>, metrics manages is started and running. Stopped, otherwise
*/
def isRunning: Boolean
}
/**
* Interface for cluster node.
*
@ -139,6 +214,8 @@ trait ClusterNode {
def reconnect(): ClusterNode
def metricsManager: NodeMetricsManager
/**
* Registers a cluster change listener.
*/

View file

@ -13,9 +13,6 @@ import akka.event.EventHandler
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteEventHandler extends Actor {
import EventHandler._
self.dispatcher = EventHandlerDispatcher
def receive = {

View file

@ -0,0 +1,82 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
/*
* {@link NodeMetricsManager} periodically refershes internal cache with node metrics from MBeans / Sigar.
* Every time local cache is refreshed, monitors plugged to the metrics manager are invoked.
* If updated metrics satisfy conditions, specified in <code>reactsOn</code>,
* <code>react</code> is called
*
* @exampl {{{
* class PeakCPULoadMonitor extends LocalMetricsAlterationMonitor {
* val id = "peak-cpu-load-monitor"
*
* def reactsOn(metrics: NodeMetrics) =
* metrics.systemLoadAverage > 0.8
*
* def react(metrics: NodeMetrics) =
* println("Peak average system load at node [%s] is reached!" format (metrics.nodeName))
* }
* }}}
*
*/
trait LocalMetricsAlterationMonitor extends MetricsAlterationMonitor {
/*
* Definies conditions that must be satisfied in order to <code>react<code> on the changed metrics
*/
def reactsOn(metrics: NodeMetrics): Boolean
/*
* Reacts on the changed metrics
*/
def react(metrics: NodeMetrics): Unit
}
/*
* {@link NodeMetricsManager} periodically refershes internal cache with metrics of all nodes in the cluster
* from ZooKeeper. Every time local cache is refreshed, monitors plugged to the metrics manager are invoked.
* If updated metrics satisfy conditions, specified in <code>reactsOn</code>,
* <code>react</code> is called
*
* @exampl {{{
* class PeakCPULoadReached extends ClusterMetricsAlterationMonitor {
* val id = "peak-cpu-load-reached"
*
* def reactsOn(metrics: Array[NodeMetrics]) =
* metrics.forall(_.systemLoadAverage > 0.8)
*
* def react(metrics: Array[NodeMetrics]) =
* println("One of the nodes in the scluster has reached the peak system load!")
* }
* }}}
*
*/
trait ClusterMetricsAlterationMonitor extends MetricsAlterationMonitor {
/*
* Definies conditions that must be satisfied in order to <code>react<code> on the changed metrics
*/
def reactsOn(allMetrics: Array[NodeMetrics]): Boolean
/*
* Reacts on the changed metrics
*/
def react(allMetrics: Array[NodeMetrics]): Unit
}
sealed trait MetricsAlterationMonitor extends Comparable[MetricsAlterationMonitor] {
/*
* Unique identiifier of the monitor
*/
def id: String
def compareTo(otherMonitor: MetricsAlterationMonitor) = id.compareTo(otherMonitor.id)
}

View file

@ -0,0 +1,44 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
/*
* Snapshot of the JVM / system that's the node is running on
*/
trait NodeMetrics {
/*
* Name of the node the metrics are gathered at
*/
def nodeName: String
/*
* Amount of heap memory currently used
*/
def usedHeapMemory: Long
/*
* Amount of heap memory guaranteed to be available
*/
def committedHeapMemory: Long
/*
* Maximum amount of heap memory that can be used
*/
def maxHeapMemory: Long
/*
* Number of the processors avalable to the JVM
*/
def avaiableProcessors: Int
/*
* If OS-specific Hyperic Sigar library is plugged, it's used to calculate
* average load on the CPUs in the system. Otherwise, value is retreived from monitoring MBeans.
* Hyperic Sigar provides more precise values, and, thus, if the library is provided, it's used by default.
*/
def systemLoadAverage: Double
}

View file

@ -4,9 +4,8 @@
package akka.dispatch
import akka.actor.{ ActorRef, Actor, IllegalActorStateException }
import util.DynamicVariable
import akka.actor.{ LocalActorRef, ActorRef, Actor, IllegalActorStateException }
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -53,15 +52,15 @@ class BalancingDispatcher(
@volatile
private var actorType: Option[Class[_]] = None
@volatile
private var members = Vector[ActorRef]()
private var members = Vector[LocalActorRef]()
private val donationInProgress = new DynamicVariable(false)
private[akka] override def register(actorRef: ActorRef) = {
private[akka] override def register(actorRef: LocalActorRef) = {
//Verify actor type conformity
actorType match {
case None actorType = Some(actorRef.actor.getClass)
case None actorType = Some(actorRef.actorInstance.get().getClass)
case Some(aType)
if (aType != actorRef.actor.getClass)
if (aType != actorRef.actorInstance.get().getClass)
throw new IllegalActorStateException(String.format(
"Can't register actor %s in a work stealing dispatcher which already knows actors of type %s",
actorRef, aType))
@ -71,7 +70,7 @@ class BalancingDispatcher(
super.register(actorRef)
}
private[akka] override def unregister(actorRef: ActorRef) = {
private[akka] override def unregister(actorRef: LocalActorRef) = {
synchronized { members = members.filterNot(actorRef eq) } //Update members
super.unregister(actorRef)
}
@ -137,7 +136,7 @@ class BalancingDispatcher(
/**
* Returns an available recipient for the message, if any
*/
protected def doFindDonorRecipient(donorMbox: MessageQueue with ExecutableMailbox, potentialRecipients: Vector[ActorRef], startIndex: Int): ActorRef = {
protected def doFindDonorRecipient(donorMbox: MessageQueue with ExecutableMailbox, potentialRecipients: Vector[LocalActorRef], startIndex: Int): ActorRef = {
val prSz = potentialRecipients.size
var i = 0
var recipient: ActorRef = null

View file

@ -7,7 +7,7 @@ package akka.dispatch
import akka.event.EventHandler
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue }
import akka.actor.{ ActorKilledException, ActorRef }
import akka.actor.{ LocalActorRef, ActorKilledException, ActorRef }
/**
* Default settings are:
@ -108,13 +108,13 @@ class Dispatcher(
/**
* @return the mailbox associated with the actor
*/
protected def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox]
protected def getMailbox(receiver: LocalActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox]
def mailboxIsEmpty(actorRef: ActorRef): Boolean = getMailbox(actorRef).isEmpty
override def mailboxIsEmpty(actorRef: LocalActorRef): Boolean = getMailbox(actorRef).isEmpty
override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
override def mailboxSize(actorRef: LocalActorRef): Int = getMailbox(actorRef).size
def createMailbox(actorRef: ActorRef): AnyRef = mailboxType match {
def createMailbox(actorRef: LocalActorRef): AnyRef = mailboxType match {
case b: UnboundedMailbox
new ConcurrentLinkedQueue[MessageInvocation] with MessageQueue with ExecutableMailbox {
@inline
@ -160,7 +160,7 @@ class Dispatcher(
private[akka] def reRegisterForExecution(mbox: MessageQueue with ExecutableMailbox): Unit =
registerForExecution(mbox)
protected override def cleanUpMailboxFor(actorRef: ActorRef) {
protected override def cleanUpMailboxFor(actorRef: LocalActorRef) {
val m = getMailbox(actorRef)
if (!m.isEmpty) {
var invocation = m.dequeue
@ -174,11 +174,10 @@ class Dispatcher(
override val toString = getClass.getSimpleName + "[" + name + "]"
def suspend(actorRef: ActorRef) {
def suspend(actorRef: LocalActorRef): Unit =
getMailbox(actorRef).suspended.tryLock
}
def resume(actorRef: ActorRef) {
def resume(actorRef: LocalActorRef): Unit = {
val mbox = getMailbox(actorRef)
mbox.suspended.tryUnlock
reRegisterForExecution(mbox)
@ -297,7 +296,7 @@ class PriorityDispatcher(
trait PriorityMailbox { self: Dispatcher
def comparator: java.util.Comparator[MessageInvocation]
override def createMailbox(actorRef: ActorRef): AnyRef = self.mailboxType match {
override def createMailbox(actorRef: LocalActorRef): AnyRef = self.mailboxType match {
case b: UnboundedMailbox
new UnboundedPriorityMessageQueue(comparator) with ExecutableMailbox {
@inline

View file

@ -7,7 +7,8 @@ package akka.dispatch
import akka.AkkaException
import akka.event.EventHandler
import akka.actor.{ Actor, ForwardableChannel, UntypedChannel, Scheduler, Timeout, ExceptionChannel }
import akka.japi.{ Procedure, Function JFunc }
import scala.Option
import akka.japi.{ Procedure, Function JFunc, Option JOption }
import scala.util.continuations._
@ -19,8 +20,7 @@ import java.util.{ LinkedList ⇒ JLinkedList }
import scala.annotation.tailrec
import scala.collection.mutable.Stack
import akka.util.{ Switch, Duration, BoxedType }
import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean }
import java.util.concurrent.atomic.{ AtomicInteger, AtomicReference, AtomicBoolean }
class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(message: String) = this(message, null)
@ -71,16 +71,45 @@ object Futures {
val futureResult = new DefaultPromise[T](timeout)
val completeFirst: Future[T] Unit = _.value.foreach(futureResult complete _)
for (f futures) f onComplete completeFirst
futures.foreach(_ onComplete completeFirst)
futureResult
}
/**
* Returns a Future that will hold the optional result of the first Future with a result that matches the predicate
*/
def find[T](predicate: T Boolean, timeout: Timeout = Timeout.default)(futures: Iterable[Future[T]]): Future[Option[T]] = {
if (futures.isEmpty) new KeptPromise[Option[T]](Right(None))
else {
val result = new DefaultPromise[Option[T]](timeout)
val ref = new AtomicInteger(futures.size)
val search: Future[T] Unit = f try {
f.result.filter(predicate).foreach(r result completeWithResult Some(r))
} finally {
if (ref.decrementAndGet == 0)
result completeWithResult None
}
futures.foreach(_ onComplete search)
result
}
}
/**
* Java API.
* Returns a Future that will hold the optional result of the first Future with a result that matches the predicate
*/
def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], timeout: Timeout): Future[JOption[T]] = {
val pred: T Boolean = predicate.apply(_)
find[T](pred, timeout)(scala.collection.JavaConversions.iterableAsScalaIterable(futures)).map(JOption.fromScalaOption(_))
}
/**
* Java API.
* Returns a Future to the result of the first future in the list that is completed
*/
def firstCompletedOf[T <: AnyRef](futures: java.lang.Iterable[Future[T]], timeout: Timeout): Future[T] =
def firstCompletedOf[T <: AnyRef](futures: JIterable[Future[T]], timeout: Timeout): Future[T] =
firstCompletedOf(scala.collection.JavaConversions.iterableAsScalaIterable(futures), timeout)
/**
@ -329,7 +358,7 @@ object Future {
try {
next.apply()
} catch {
case e // FIXME
case e // TODO FIXME: Throwable or Exception, log or do what?
}
}
} finally { _taskStack set None }
@ -645,7 +674,7 @@ package japi {
/* Java API */
trait Future[+T] { self: akka.dispatch.Future[T]
private[japi] final def onTimeout[A >: T](proc: Procedure[akka.dispatch.Future[A]]): this.type = self.onTimeout(proc(_))
private[japi] final def onResult[A >: T](proc: Procedure[A]): this.type = self.onResult({ case r: A proc(r) }: PartialFunction[T, Unit])
private[japi] final def onResult[A >: T](proc: Procedure[A]): this.type = self.onResult({ case r proc(r.asInstanceOf[A]) }: PartialFunction[T, Unit])
private[japi] final def onException(proc: Procedure[Throwable]): this.type = self.onException({ case t: Throwable proc(t) }: PartialFunction[Throwable, Unit])
private[japi] final def onComplete[A >: T](proc: Procedure[akka.dispatch.Future[A]]): this.type = self.onComplete(proc(_))
private[japi] final def map[A >: T, B](f: JFunc[A, B]): akka.dispatch.Future[B] = self.map(f(_))

View file

@ -16,7 +16,7 @@ import akka.actor._
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
final case class MessageInvocation(val receiver: ActorRef,
final case class MessageInvocation(val receiver: LocalActorRef,
val message: Any,
val channel: UntypedChannel) {
if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null")
@ -63,7 +63,7 @@ abstract class MessageDispatcher {
/**
* Creates and returns a mailbox for the given actor.
*/
private[akka] def createMailbox(actorRef: ActorRef): AnyRef
private[akka] def createMailbox(actorRef: LocalActorRef): AnyRef
/**
* Name of this dispatcher.
@ -73,7 +73,7 @@ abstract class MessageDispatcher {
/**
* Attaches the specified actorRef to this dispatcher
*/
final def attach(actorRef: ActorRef) {
final def attach(actorRef: LocalActorRef) {
guard withGuard {
register(actorRef)
}
@ -82,7 +82,7 @@ abstract class MessageDispatcher {
/**
* Detaches the specified actorRef from this dispatcher
*/
final def detach(actorRef: ActorRef) {
final def detach(actorRef: LocalActorRef) {
guard withGuard {
unregister(actorRef)
}
@ -129,7 +129,7 @@ abstract class MessageDispatcher {
* Only "private[akka] for the sake of intercepting calls, DO NOT CALL THIS OUTSIDE OF THE DISPATCHER,
* and only call it under the dispatcher-guard, see "attach" for the only invocation
*/
private[akka] def register(actorRef: ActorRef) {
private[akka] def register(actorRef: LocalActorRef) {
if (actorRef.mailbox eq null)
actorRef.mailbox = createMailbox(actorRef)
@ -145,7 +145,7 @@ abstract class MessageDispatcher {
* Only "private[akka] for the sake of intercepting calls, DO NOT CALL THIS OUTSIDE OF THE DISPATCHER,
* and only call it under the dispatcher-guard, see "detach" for the only invocation
*/
private[akka] def unregister(actorRef: ActorRef) = {
private[akka] def unregister(actorRef: LocalActorRef) = {
if (uuids remove actorRef.uuid) {
cleanUpMailboxFor(actorRef)
actorRef.mailbox = null
@ -166,7 +166,7 @@ abstract class MessageDispatcher {
* Overridable callback to clean up the mailbox for a given actor,
* called when an actor is unregistered.
*/
protected def cleanUpMailboxFor(actorRef: ActorRef) {}
protected def cleanUpMailboxFor(actorRef: LocalActorRef) {}
/**
* Traverses the list of actors (uuids) currently being attached to this dispatcher and stops those actors
@ -211,12 +211,12 @@ abstract class MessageDispatcher {
/**
* After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference
*/
def suspend(actorRef: ActorRef)
def suspend(actorRef: LocalActorRef)
/*
* After the call to this method, the dispatcher must begin any new message processing for the specified reference
*/
def resume(actorRef: ActorRef)
def resume(actorRef: LocalActorRef)
/**
* Will be called when the dispatcher is to queue an invocation for execution
@ -238,12 +238,12 @@ abstract class MessageDispatcher {
/**
* Returns the size of the mailbox for the specified actor
*/
def mailboxSize(actorRef: ActorRef): Int
def mailboxSize(actorRef: LocalActorRef): Int
/**
* Returns the "current" emptiness status of the mailbox for the specified actor
*/
def mailboxIsEmpty(actorRef: ActorRef): Boolean
def mailboxIsEmpty(actorRef: LocalActorRef): Boolean
/**
* Returns the amount of tasks queued for execution

View file

@ -5,7 +5,7 @@
package akka.dispatch
import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorRef
import akka.actor.{ LocalActorRef, ActorRef }
/**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
@ -33,14 +33,14 @@ class PinnedDispatcher(_actor: ActorRef, _name: String, _mailboxType: MailboxTyp
private[akka] val owner = new AtomicReference[ActorRef](_actor)
//Relies on an external lock provided by MessageDispatcher.attach
private[akka] override def register(actorRef: ActorRef) = {
private[akka] override def register(actorRef: LocalActorRef) = {
val actor = owner.get()
if ((actor ne null) && actorRef != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor)
owner.compareAndSet(null, actorRef) //Register if unregistered
super.register(actorRef)
}
//Relies on an external lock provided by MessageDispatcher.detach
private[akka] override def unregister(actorRef: ActorRef) = {
private[akka] override def unregister(actorRef: LocalActorRef) = {
super.unregister(actorRef)
owner.compareAndSet(actorRef, null) //Unregister (prevent memory leak)
}

View file

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

View file

@ -1,5 +1,7 @@
package akka.japi
import scala.Some
/**
* A Function interface. Used to create first-class-functions is Java (sort of).
*/
@ -84,6 +86,14 @@ object Option {
*/
def option[A](v: A): Option[A] = if (v == null) none else some(v)
/**
* Converts a Scala Option to a Java Option
*/
def fromScalaOption[T](scalaOption: scala.Option[T]): Option[T] = scalaOption match {
case scala.Some(r) some(r)
case scala.None none
}
/**
* Class <code>Some[A]</code> represents existing values of type
* <code>A</code>.
@ -104,5 +114,5 @@ object Option {
}
implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala
implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = option(o.get)
implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = if (o.isDefined) some(o.get) else none
}

View file

@ -4,9 +4,9 @@
package akka.routing
import akka.actor.{ Actor, ActorRef, PoisonPill, Death, MaximumNumberOfRestartsWithinTimeRangeReached }
import akka.dispatch.{ Promise }
import akka.config.Supervision._
import akka.actor._
/**
* Actor pooling
@ -59,28 +59,6 @@ trait ActorPool {
def select(delegates: Seq[ActorRef]): Seq[ActorRef]
}
/**
* Defines the configuration options for how the pool supervises the actors.
*/
trait ActorPoolSupervisionConfig {
/**
* Defines the default fault handling strategy to be employed by the pool.
*/
def poolFaultHandler: FaultHandlingStrategy
}
/**
* Provides a default implementation of the supervision configuration by
* defining a One-for-One fault handling strategy, trapping exceptions,
* limited to 5 retries within 1 second.
*
* This is just a basic strategy and implementors are encouraged to define
* something more appropriate for their needs.
*/
trait DefaultActorPoolSupervisionConfig extends ActorPoolSupervisionConfig {
def poolFaultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000)
}
/**
* A default implementation of a pool that:
* First, invokes the pool's capacitor that tells it, based on the current delegate count
@ -104,14 +82,11 @@ trait DefaultActorPoolSupervisionConfig extends ActorPoolSupervisionConfig {
*
* Lastly, routes by forwarding, the incoming message to each delegate in the selected set.
*/
trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervisionConfig
trait DefaultActorPool extends ActorPool { this: Actor
import ActorPool._
protected[akka] var _delegates = Vector[ActorRef]()
override def preStart() {
self.faultHandler = poolFaultHandler
}
override def postStop() {
_delegates foreach { delegate
try {
@ -141,8 +116,8 @@ trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervision
_delegates ++ {
for (i 0 until requestedCapacity) yield {
val delegate = instance()
self startLink delegate
delegate
self link delegate
delegate.start()
}
}
case qty if qty < 0
@ -176,8 +151,13 @@ trait SmallestMailboxSelector {
var set: Seq[ActorRef] = Nil
var take = if (partialFill) math.min(selectionCount, delegates.length) else selectionCount
def mailboxSize(a: ActorRef): Int = a match {
case l: LocalActorRef l.dispatcher.mailboxSize(l)
case _ Int.MaxValue //Non-local actors mailbox size is unknown, so consider them lowest priority
}
while (take > 0) {
set = delegates.sortWith((a, b) a.dispatcher.mailboxSize(a) < b.dispatcher.mailboxSize(b)).take(take) ++ set //Question, doesn't this risk selecting the same actor multiple times?
set = delegates.sortWith((a, b) mailboxSize(a) < mailboxSize(b)).take(take) ++ set //Question, doesn't this risk selecting the same actor multiple times?
take -= set.size
}
@ -257,7 +237,10 @@ trait BoundedCapacitor {
trait MailboxPressureCapacitor {
def pressureThreshold: Int
def pressure(delegates: Seq[ActorRef]): Int =
delegates count { a a.dispatcher.mailboxSize(a) > pressureThreshold }
delegates count {
case a: LocalActorRef a.dispatcher.mailboxSize(a) > pressureThreshold
case _ false
}
}
/**
@ -265,7 +248,10 @@ trait MailboxPressureCapacitor {
*/
trait ActiveFuturesPressureCapacitor {
def pressure(delegates: Seq[ActorRef]): Int =
delegates count { _.channel.isInstanceOf[Promise[_]] }
delegates count {
case fc: ForwardableChannel fc.channel.isInstanceOf[Promise[_]]
case _ false
}
}
/**

View file

@ -7,18 +7,14 @@ package akka.routing
import annotation.tailrec
import akka.AkkaException
import akka.dispatch.Future
import akka.actor._
import akka.dispatch.Futures
import akka.event.EventHandler
import akka.actor.UntypedChannel._
import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger }
/**
* An {@link AkkaException} thrown when something goes wrong while routing a message
*/
class RoutingException(message: String) extends AkkaException(message)
import akka.dispatch.{ Future, Futures }
import akka.util.ReflectiveAccess
import collection.JavaConversions.iterableAsScalaIterable
sealed trait RouterType
@ -65,6 +61,116 @@ object RouterType {
}
object RoutedProps {
final val defaultTimeout = Actor.TIMEOUT
final val defaultRouterFactory = () new RoundRobinRouter
final val defaultDeployId = ""
final val defaultLocalOnly = !ReflectiveAccess.ClusterModule.isEnabled
/**
* The default RoutedProps instance, uses the settings from the RoutedProps object starting with default*
*/
final val default = new RoutedProps()
def apply(): RoutedProps = default
}
/**
* Contains the configuration to create local and clustered routed actor references.
*
* Routed ActorRef configuration object, this is thread safe and fully sharable.
*
* Because the Routers are stateful, a new Router instance needs to be created for every ActorRef that relies on routing
* (currently the ClusterActorRef and the RoutedActorRef). That is why a Router factory is used (a function that returns
* a new Router instance) instead of a single Router instance. This makes sharing the same RoutedProps between multiple
* threads safe.
*
* This configuration object makes it possible to either
*/
case class RoutedProps(routerFactory: () Router, deployId: String, connections: Iterable[ActorRef], timeout: Timeout, localOnly: Boolean) {
def this() = this(
routerFactory = RoutedProps.defaultRouterFactory,
deployId = RoutedProps.defaultDeployId,
connections = List(),
timeout = RoutedProps.defaultTimeout,
localOnly = RoutedProps.defaultLocalOnly)
/**
* Returns a new RoutedProps with the specified deployId set
*
* Java and Scala API
*/
def withDeployId(id: String): RoutedProps = copy(deployId = if (id eq null) "" else id)
/**
* Returns a new RoutedProps configured with a random router.
*
* Java and Scala API.
*/
def withRandomRouter(): RoutedProps = copy(routerFactory = () new RandomRouter())
/**
* Returns a new RoutedProps configured with a round robin router.
*
* Java and Scala API.
*/
def withRoundRobinRouter(): RoutedProps = copy(routerFactory = () new RoundRobinRouter())
/**
* Returns a new RoutedProps configured with a direct router.
*
* Java and Scala API.
*/
def withDirectRouter(): RoutedProps = copy(routerFactory = () new DirectRouter())
/**
* Makes it possible to change the default behavior in a clustered environment that a clustered actor ref is created.
* In some cases you just want to have local actor references, even though the Cluster Module is up and running.
*
* Java and Scala API.
*/
def withLocalOnly(l: Boolean = true) = copy(localOnly = l)
/**
* Sets the Router factory method to use. Since Router instance contain state, and should be linked to a single 'routed' ActorRef, a new
* Router instance is needed for every 'routed' ActorRef. That is why a 'factory' function is used to create new
* instances.
*
* Scala API.
*/
def withRouter(f: () Router): RoutedProps = copy(routerFactory = f)
/**
* Sets the RouterFactory to use. Since Router instance contain state, and should be linked to a single 'routed' ActorRef, a new
* Router instance is needed for every 'routed' ActorRef. That is why a RouterFactory interface is used to create new
* instances.
*
* Java API.
*/
def withRouter(f: RouterFactory): RoutedProps = copy(routerFactory = () f.newRouter())
/**
*
*/
def withTimeout(t: Timeout): RoutedProps = copy(timeout = t)
/**
* Sets the connections to use.
*
* Scala API.
*/
def withConnections(c: Iterable[ActorRef]): RoutedProps = copy(connections = c)
/**
* Sets the connections to use.
*
* Java API.
*/
def withConnections(c: java.lang.Iterable[ActorRef]): RoutedProps = copy(connections = iterableAsScalaIterable(c))
}
/**
* The Router is responsible for sending a message to one (or more) of its connections. Connections are stored in the
* {@link RouterConnections} and each Router should be linked to only one {@link RouterConnections}.
@ -102,7 +208,24 @@ trait Router {
}
/**
* An {@link AkkaException} thrown when something goes wrong while routing a message
*/
class RoutingException(message: String) extends AkkaException(message)
/**
* The RouterConnection acts like a middleman between the Router and the actor reference that does the routing.
* Through the RouterConnection:
* <ol>
* <li>
* the actor ref can signal that something has changed in the known set of connections. The Router can see
* when a changed happened (by checking the version) and update its internal datastructures.
* </li>
* <li>
* the Router can indicate that some happened happened with a actor ref, e.g. the actor ref dying.
* </li>
* </ol>
*
* It is very likely that the implementation of the RouterConnection will be part of the ActorRef itself.
*/
trait RouterConnections {
@ -113,17 +236,20 @@ trait RouterConnections {
def version: Long
/**
* Returns the number of connections.
* Returns the number of connections. Value could be stale as soon as received, and this method can't be combined (easily)
* with an atomic read of and size and version.
*/
def size: Int
/**
* Returns a tuple containing the version and Iterable of all connected ActorRefs this Router uses to send messages to.
* Returns a VersionedIterator containing all connectected ActorRefs at some moment in time. Since there is
* the time element, also the version is included to be able to read the data (the connections) and the version
* in an atomic manner.
*
* This iterator should be 'persistent'. So it can be handed out to other threads so that they are working on
* a stable (immutable) view of some set of connections.
* This Iterable is 'persistent'. So it can be handed out to different threads and they see a stable (immutable)
* view of some set of connections.
*/
def versionedIterator: (Long, Iterable[ActorRef])
def versionedIterable: VersionedIterable[ActorRef]
/**
* A callback that can be used to indicate that a connected actorRef was dead.
@ -135,19 +261,51 @@ trait RouterConnections {
*
* It could be that even after a remove has been called for a specific ActorRef, that the ActorRef
* is still being used. A good behaving Router will eventually discard this reference, but no guarantees are
* made how long this takes place.
* made how long this takes.
*
* @param ref the dead
*/
def remove(deadRef: ActorRef): Unit
}
/**
* An Iterable that also contains a version.
*/
case class VersionedIterable[A](version: Long, val iterable: Iterable[A])
/**
* A Helper class to create actor references that use routing.
*/
object Routing {
sealed trait RoutingMessage
case class Broadcast(message: Any) extends RoutingMessage
/**
* todo: will very likely be moved to the ActorRef.
*/
def actorOf(props: RoutedProps): ActorRef = {
//TODO Implement support for configuring by deployment ID etc
//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?)
//TODO If the actor deployed uses a different config, then ignore or throw exception?
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()
} else {
if (props.connections.isEmpty)
throw new IllegalArgumentException("A routed actorRef can't have an empty connection set")
new RoutedActorRef(props).start()
}
}
/**
* Creates a new started RoutedActorRef that uses routing to deliver a message to one of its connected actors.
*
@ -157,44 +315,37 @@ object Routing {
* @throws IllegalArgumentException if the number of connections is zero, or if it depends on the actual router implementation
* how many connections it can handle.
*/
@deprecated
def actorOf(actorAddress: String, connections: Iterable[ActorRef], routerType: RouterType): ActorRef = {
val ref = routerType match {
val router = routerType match {
case RouterType.Direct if connections.size > 1
throw new IllegalArgumentException("A direct router can't have more than 1 connection")
case RouterType.Direct
if (connections.size > 1)
throw new IllegalArgumentException("A direct router can't have more than 1 connection")
actorOf(actorAddress, connections, new DirectRouter())
new DirectRouter()
case RouterType.Random
actorOf(actorAddress, connections, new RandomRouter())
new RandomRouter()
case RouterType.RoundRobin
actorOf(actorAddress, connections, new RoundRobinRouter())
case _ throw new IllegalArgumentException("Unsupported routerType " + routerType)
new RoundRobinRouter()
case r
throw new IllegalArgumentException("Unsupported routerType " + r)
}
ref.start()
}
def actorOf(actorAddress: String, connections: Iterable[ActorRef], router: Router): ActorRef = {
if (connections.size == 0)
throw new IllegalArgumentException("To create a routed actor ref, at least one connection is required")
new RoutedActorRef(actorAddress, router, connections)
}
def actorOfWithRoundRobin(actorAddress: String, connections: Iterable[ActorRef]): ActorRef = {
actorOf(actorAddress, connections, akka.routing.RouterType.RoundRobin)
val props = new RoutedProps(() router, actorAddress, connections, RoutedProps.defaultTimeout, true)
new RoutedActorRef(props).start()
}
}
/**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors.
* An Abstract convenience implementation for building an ActorReference that uses a Router.
*/
class RoutedActorRef(val address: String, val router: Router, val connectionIterator: Iterable[ActorRef]) extends UnsupportedActorRef {
abstract private[akka] class AbstractRoutedActorRef(val props: RoutedProps) extends UnsupportedActorRef {
router.init(new RoutedActorRefConnections(connectionIterator))
val router = props.routerFactory.apply()
def address = props.deployId
override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = {
val sender = channel match {
@ -212,9 +363,20 @@ class RoutedActorRef(val address: String, val router: Router, val connectionIter
}
router.route[Any](message, timeout)(sender)
}
}
/**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors.
*/
private[akka] class RoutedActorRef(val routedProps: RoutedProps)
extends AbstractRoutedActorRef(routedProps) {
router.init(new RoutedActorRefConnections(routedProps.connections))
def start(): this.type = synchronized[this.type] {
_status = ActorRefInternals.RUNNING
if (_status == ActorRefInternals.UNSTARTED)
_status = ActorRefInternals.RUNNING
this
}
@ -223,49 +385,40 @@ class RoutedActorRef(val address: String, val router: Router, val connectionIter
if (_status == ActorRefInternals.RUNNING) {
_status = ActorRefInternals.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
// FIXME here we need to fire off Actor.cluster.remove(address) (which needs to be properly implemented first, see ticket)
//inetSocketAddressToActorRefMap.get.values foreach (_.stop()) // shut down all remote connections
}
}
}
private class RoutedActorRefConnections() extends RouterConnections {
private val state = new AtomicReference[State]()
private val state = new AtomicReference[VersionedIterable[ActorRef]]()
def this(connectionIterable: Iterable[ActorRef]) = {
this()
state.set(new State(Long.MinValue, connectionIterable))
state.set(new VersionedIterable[ActorRef](Long.MinValue, connectionIterable))
}
def version: Long = state.get().version
def size: Int = state.get().connections.size
def size: Int = state.get().iterable.size
def versionedIterator = {
val s = state.get
(s.version, s.connections)
}
def versionedIterable = state.get
@tailrec
final def remove(ref: ActorRef) = {
val oldState = state.get()
//remote the ref from the connections.
var newList = oldState.connections.filter(currentActorRef currentActorRef ne ref)
var newList = oldState.iterable.filter(currentActorRef currentActorRef ne ref)
if (newList.size != oldState.connections.size) {
if (newList.size != oldState.iterable.size) {
//one or more occurrences of the actorRef were removed, so we need to update the state.
val newState = new State(oldState.version + 1, newList)
val newState = new VersionedIterable[ActorRef](oldState.version + 1, newList)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) remove(ref)
}
}
case class State(val version: Long, val connections: Iterable[ActorRef])
}
}
@ -288,7 +441,7 @@ trait BasicRouter extends Router {
def route(message: Any)(implicit sender: Option[ActorRef]): Unit = message match {
case Routing.Broadcast(message)
//it is a broadcast message, we are going to send to message to all connections.
connections.versionedIterator._2.foreach(actor
connections.versionedIterable.iterable.foreach(actor
try {
actor.!(message)(sender)
} catch {
@ -341,7 +494,8 @@ trait BasicRouter extends Router {
}
/**
* A DirectRouter is FIXME
* A DirectRouter a Router that only has a single connected actorRef and forwards all request to that actorRef.
*
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
@ -364,19 +518,18 @@ class DirectRouter extends BasicRouter {
} else {
//there has been a change in the connections, or this is the first time this method is called. So we are going to do some updating.
val (version, connectionIterable) = connections.versionedIterator
val versionedIterable = connections.versionedIterable
if (connectionIterable.size > 1)
throw new RoutingException("A DirectRouter can't have more than 1 connected Actor, but found [%s]".format(connectionIterable.size))
val connectionCount = versionedIterable.iterable.size
if (connectionCount > 1)
throw new RoutingException("A DirectRouter can't have more than 1 connected Actor, but found [%s]".format(connectionCount))
val newState = new DirectRouterState(connectionIterable.head, version)
if (state.compareAndSet(currentState, newState)) {
val newState = new DirectRouterState(versionedIterable.iterable.head, versionedIterable.version)
if (state.compareAndSet(currentState, newState))
//we are lucky since we just updated the state, so we can send it back as the state to use
newState
} else {
//we failed to update the state, lets try again... better luck next time.
else //we failed to update the state, lets try again... better luck next time.
getState()
}
}
}
@ -394,16 +547,11 @@ class RandomRouter extends BasicRouter {
private val state = new AtomicReference[RandomRouterState]()
//FIXME: threadlocal random?
private val random = new java.util.Random(System.currentTimeMillis)
private val random = new java.util.Random(System.nanoTime())
def next: Option[ActorRef] = {
val state = getState()
if (state.array.isEmpty) {
None
} else {
val index = random.nextInt(state.array.length)
Some(state.array(index))
}
def next: Option[ActorRef] = getState().array match {
case a if a.isEmpty None
case a Some(a(random.nextInt(a.length)))
}
@tailrec
@ -415,20 +563,18 @@ class RandomRouter extends BasicRouter {
currentState
} else {
//there has been a change in connections, or it was the first try, so we need to update the internal state
val (version, connectionIterable) = connections.versionedIterator
val newState = new RandomRouterState(connectionIterable.toArray[ActorRef], version)
if (state.compareAndSet(currentState, newState)) {
val versionedIterable = connections.versionedIterable
val newState = new RandomRouterState(versionedIterable.iterable.toIndexedSeq, versionedIterable.version)
if (state.compareAndSet(currentState, newState))
//we are lucky since we just updated the state, so we can send it back as the state to use
newState
} else {
//we failed to update the state, lets try again... better luck next time.
else //we failed to update the state, lets try again... better luck next time.
getState()
}
}
}
private case class RandomRouterState(val array: Array[ActorRef], val version: Long)
private case class RandomRouterState(array: IndexedSeq[ActorRef], version: Long)
}
/**
@ -451,20 +597,18 @@ class RoundRobinRouter extends BasicRouter {
currentState
} else {
//there has been a change in connections, or it was the first try, so we need to update the internal state
val (version, connectionIterable) = connections.versionedIterator
val newState = new RoundRobinState(connectionIterable.toArray[ActorRef], version)
if (state.compareAndSet(currentState, newState)) {
val versionedIterable = connections.versionedIterable
val newState = new RoundRobinState(versionedIterable.iterable.toIndexedSeq[ActorRef], versionedIterable.version)
if (state.compareAndSet(currentState, newState))
//we are lucky since we just updated the state, so we can send it back as the state to use
newState
} else {
//we failed to update the state, lets try again... better luck next time.
else //we failed to update the state, lets try again... better luck next time.
getState()
}
}
}
private case class RoundRobinState(val array: Array[ActorRef], val version: Long) {
private case class RoundRobinState(array: IndexedSeq[ActorRef], version: Long) {
private val index = new AtomicInteger(0)
@ -500,7 +644,7 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G]
private def scatterGather[S, G >: S](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[G] = {
val responses = connections.versionedIterator._2.flatMap { actor
val responses = connections.versionedIterable.iterable.flatMap { actor
try {
Some(actor.?(message, timeout)(sender).asInstanceOf[Future[S]])
} catch {
@ -510,10 +654,10 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
}
}
if (responses.size == 0)
if (responses.isEmpty)
throw new RoutingException("No connections can process the message [%s] sent to scatter-gather router" format (message))
gather(responses)
else
gather(responses)
}
override def route[T](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[T] = message match {

View file

@ -13,6 +13,7 @@ import akka.event.EventHandler
import akka.cluster.ClusterNode
import java.net.InetSocketAddress
import akka.routing.{ RoutedProps, Router }
/**
* Helper class for reflective access to different modules in order to allow optional loading of modules.
@ -33,6 +34,21 @@ object ReflectiveAccess {
object ClusterModule {
lazy val isEnabled = Config.isClusterEnabled //&& clusterInstance.isDefined
lazy val clusterRefClass: Class[_] = getClassFor("akka.cluster.ClusterActorRef") match {
case Left(e) throw e
case Right(b) b
}
def newClusteredActorRef(props: RoutedProps): ActorRef = {
val params: Array[Class[_]] = Array(classOf[RoutedProps])
val args: Array[AnyRef] = Array(props)
createInstance(clusterRefClass, params, args) match {
case Left(e) throw e
case Right(b) b.asInstanceOf[ActorRef]
}
}
def ensureEnabled() {
if (!isEnabled) {
val e = new ModuleNotAvailableException(

View file

@ -7,7 +7,7 @@ package akka.camel
import java.lang.reflect.Method
import java.lang.reflect.Proxy._
import akka.actor.{ TypedActor, ActorRef }
import akka.actor.{ LocalActorRef, TypedActor, ActorRef }
import akka.actor.TypedActor._
/**
@ -42,10 +42,11 @@ private[camel] object TypedConsumer {
if (m.isAnnotationPresent(classOf[consume]))
} yield f(tc, m)
private def withConsumeAnnotatedMethodsonImplClass[T](tc: AnyRef, actorRef: ActorRef, f: (AnyRef, Method) T): List[T] = {
val implClass = actorRef.actor.asInstanceOf[TypedActor.TypedActor[AnyRef, AnyRef]].me.getClass
for (m implClass.getDeclaredMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(tc, m)
private def withConsumeAnnotatedMethodsonImplClass[T](tc: AnyRef, actorRef: ActorRef, f: (AnyRef, Method) T): List[T] = actorRef match {
case l: LocalActorRef
val implClass = l.actorInstance.get().asInstanceOf[TypedActor.TypedActor[AnyRef, AnyRef]].me.getClass
for (m implClass.getDeclaredMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(tc, m)
case _ Nil
}
private class ProxyClass(c: Class[_]) {

View file

@ -139,12 +139,12 @@ private[camel] object Consumer {
* to a consumer actor, <code>Some</code> contained the return value of <code>f</code>
* otherwise.
*/
def withConsumer[T](actorRef: ActorRef)(f: Consumer T): Option[T] = {
if (!actorRef.actor.isInstanceOf[Consumer]) None
// TODO: check if this is needed at all
//else if (actorRef.homeAddress.isDefined) None
else Some(f(actorRef.actor.asInstanceOf[Consumer]))
def withConsumer[T](actorRef: ActorRef)(f: Consumer T): Option[T] = actorRef match {
case l: LocalActorRef
l.actorInstance.get() match {
case c: Consumer Some(f(c))
case _ None
}
case _ None
}
}

View file

@ -291,37 +291,16 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
callback.done(false)
}
def actorClass: Class[_ <: Actor] = unsupported
def dispatcher_=(md: MessageDispatcher): Unit = unsupported
def dispatcher: MessageDispatcher = unsupported
def makeRemote(hostname: String, port: Int): Unit = unsupported
def makeRemote(address: InetSocketAddress): Unit = unsupported
def homeAddress_=(address: InetSocketAddress): Unit = unsupported
def remoteAddress: Option[InetSocketAddress] = unsupported
def link(actorRef: ActorRef): Unit = unsupported
def unlink(actorRef: ActorRef): Unit = unsupported
def startLink(actorRef: ActorRef): ActorRef = unsupported
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = unsupported
def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported
def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = unsupported
def link(actorRef: ActorRef): ActorRef = unsupported
def unlink(actorRef: ActorRef): ActorRef = unsupported
def shutdownLinkedActors: Unit = unsupported
def supervisor: Option[ActorRef] = unsupported
def homeAddress: Option[InetSocketAddress] = None
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(message: Any, timeout: Timeout, channel: UntypedChannel) = unsupported
protected[akka] def mailbox: AnyRef = unsupported
protected[akka] def mailbox_=(msg: AnyRef): AnyRef = unsupported
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
def linkedActors: JMap[Uuid, ActorRef] = unsupported
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
protected[akka] def registerSupervisorAsRemoteActor = unsupported
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName)
}

View file

@ -28,7 +28,6 @@ object CamelTestSupport {
}
class Tester3 extends TestActor with Noop {
self.timeout = 1
def handler = noop
}

View file

@ -14,7 +14,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite {
var publisher: ActorRef = _
var requestor: ActorRef = _
var consumer: ActorRef = _
var consumer: LocalActorRef = _
@Before
def setUp: Unit = {
@ -24,7 +24,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite {
consumer = actorOf(new Actor with Consumer {
def endpointUri = "mock:test"
protected def receive = null
}).start
}).start.asInstanceOf[LocalActorRef]
}
@After
@ -39,7 +39,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite {
requestor ! ActorRegistered(consumer.address, consumer, None)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
assert((publisher ? GetRetainedMessage).get ===
ConsumerActorRegistered(consumer, consumer.actor.asInstanceOf[Consumer]))
ConsumerActorRegistered(consumer, consumer.actorInstance.get.asInstanceOf[Consumer]))
}
@Test
@ -48,7 +48,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite {
requestor ! ActorUnregistered(consumer.address, consumer, None)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
assert((publisher ? GetRetainedMessage).get ===
ConsumerActorUnregistered(consumer, consumer.actor.asInstanceOf[Consumer]))
ConsumerActorUnregistered(consumer, consumer.actorInstance.get.asInstanceOf[Consumer]))
}
}

View file

@ -2,7 +2,7 @@ package akka.camel
import org.junit.Test
import org.scalatest.junit.JUnitSuite
import akka.actor.{ ActorRef, Actor }
import akka.actor.{ ActorRef, Actor, LocalActorRef }
class ConsumerRegisteredTest extends JUnitSuite {
import ConsumerRegisteredTest._
@ -48,7 +48,10 @@ class ConsumerRegisteredTest extends JUnitSuite {
assert(event === None)
}
private def consumerOf(ref: ActorRef) = ref.actor.asInstanceOf[Consumer]
private def consumerOf(ref: ActorRef) = ref match {
case l: LocalActorRef l.actorInstance.get.asInstanceOf[Consumer]
case _ null: Consumer
}
}
object ConsumerRegisteredTest {

View file

@ -103,7 +103,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(new TestBlocker("direct:publish-test-5")).start
actorOf(Props(creator = () new TestBlocker("direct:publish-test-5"), timeout = Timeout(1000))).start
} must be(true)
try {
@ -188,7 +188,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher
}
"be able to reply on failure during postStop" in {
val consumer = Actor.actorOf(new SupervisedConsumer("reply-channel-test-3"))
val consumer = Actor.actorOf(Props(new SupervisedConsumer("reply-channel-test-3")).withLifeCycle(Temporary))
val supervisor = Supervisor(
SupervisorConfig(
OneForOneStrategy(List(classOf[Exception]), 2, 10000),
@ -216,7 +216,6 @@ object ConsumerScalaTest {
}
class TestBlocker(uri: String) extends Actor with BlockingConsumer {
self.timeout = 1000
def endpointUri = uri
protected def receive = {
case msg: Message { /* do not reply */ }

View file

@ -95,11 +95,10 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
when("a test message is sent to the producer with ?")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = (producer ? message).get
val result = (producer ? message).as[Message].get
then("a normal response should have been returned by the producer")
val expected = Message("received test", Map(Message.MessageExchangeId -> "123"))
assert(result === expected)
assert(result.headers(Message.MessageExchangeId) === "123")
}
scenario("produce message and receive failure response") {
@ -188,11 +187,11 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
when("a test message is sent to the producer with ?")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = (producer ? message).get
val result = (producer ? message).as[Message].get
then("a normal response should have been returned by the forward target")
val expected = Message("received test", Map(Message.MessageExchangeId -> "123", "test" -> "result"))
assert(result === expected)
assert(result.headers(Message.MessageExchangeId) === "123")
assert(result.headers("test") === "result")
}
scenario("produce message, forward failure response to a replying target actor and receive response") {

View file

@ -7,7 +7,7 @@ import org.apache.camel.builder.RouteBuilder
import org.apache.camel.component.mock.MockEndpoint
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec }
import akka.actor.Actor
import akka.actor.{ Actor, Props, Timeout }
import akka.actor.Actor._
import akka.camel.{ Failure, Message, CamelContextManager }
import akka.camel.CamelTestSupport._
@ -47,7 +47,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
}
scenario("two-way communication with timeout") {
val actor = actorOf[Tester3].start
val actor = actorOf(Props[Tester3].withTimeout(Timeout(1))).start
intercept[RuntimeCamelException] {
mandatoryTemplate.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin")
}

View file

@ -10,6 +10,7 @@ import org.junit.{ After, Test }
import org.scalatest.junit.JUnitSuite
import org.scalatest.BeforeAndAfterAll
import akka.actor.{ Props, Timeout }
import akka.actor.Actor._
import akka.camel.{ Failure, Message }
import akka.camel.CamelTestSupport._
@ -228,7 +229,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
@Test
def shouldSendMessageToActorAndTimeout(): Unit = {
val actor = actorOf[Tester3].start
val actor = actorOf(Props[Tester3].withTimeout(Timeout(1))).start
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")

View file

@ -15,12 +15,11 @@ import org.I0Itec.zkclient.exception._
import java.util.{ List JList }
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference }
import java.net.InetSocketAddress
import javax.management.StandardMBean
import scala.collection.mutable.ConcurrentMap
import scala.collection.JavaConversions._
import akka.util._
import duration._
import Helpers._
import akka.actor._
@ -29,8 +28,7 @@ import Status._
import DeploymentConfig._
import akka.event.EventHandler
import akka.dispatch.{ Dispatchers, Future }
import akka.cluster._
import akka.dispatch.{ Dispatchers, Future, PinnedDispatcher }
import akka.cluster._
import akka.routing.RouterType
@ -42,6 +40,7 @@ import akka.serialization.{ Serialization, Serializer, ActorSerialization }
import ActorSerialization._
import akka.serialization.Compression.LZF
import akka.cluster.metrics._
import akka.cluster.zookeeper._
import ChangeListener._
import ClusterProtocol._
@ -54,6 +53,7 @@ import com.google.protobuf.ByteString
import java.util.concurrent.{ CopyOnWriteArrayList, Callable, ConcurrentHashMap }
import annotation.tailrec
import javax.management.{ StandardMBean }
// FIXME add watch for each node that when the entry for the node is removed then the node shuts itself down
@ -152,6 +152,7 @@ object Cluster {
val zooKeeperServers = config.getString("akka.cluster.zookeeper-server-addresses", "localhost:2181")
val remoteServerPort = config.getInt("akka.cluster.remote-server-port", 2552)
val sessionTimeout = Duration(config.getInt("akka.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt
val metricsRefreshInterval = Duration(config.getInt("akka.cluster.metrics-refresh-timeout", 2), TIME_UNIT)
val connectionTimeout = Duration(config.getInt("akka.cluster.connection-timeout", 60), TIME_UNIT).toMillis.toInt
val maxTimeToWaitUntilConnected = Duration(config.getInt("akka.cluster.max-time-to-wait-until-connected", 30), TIME_UNIT).toMillis.toInt
val shouldCompressData = config.getBool("akka.cluster.use-compression", false)
@ -278,15 +279,15 @@ class DefaultClusterNode private[akka] (
// private val connectToAllNewlyArrivedMembershipNodesInClusterLock = new AtomicBoolean(false)
private[cluster] lazy val remoteClientLifeCycleHandler = localActorOf(new Actor {
private[cluster] lazy val remoteClientLifeCycleHandler = actorOf(Props(new Actor {
def receive = {
case RemoteClientError(cause, client, address) client.shutdownClientModule()
case RemoteClientDisconnected(client, address) client.shutdownClientModule()
case _ //ignore other
}
}, "akka.cluster.RemoteClientLifeCycleListener").start()
}), "akka.cluster.RemoteClientLifeCycleListener")
private[cluster] lazy val remoteDaemon = localActorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.Address).start()
private[cluster] lazy val remoteDaemon = actorOf(Props(new RemoteClusterDaemon(this)).copy(dispatcher = new PinnedDispatcher(), localOnly = true), RemoteClusterDaemon.Address)
private[cluster] lazy val remoteDaemonSupervisor = Supervisor(
SupervisorConfig(
@ -307,6 +308,8 @@ class DefaultClusterNode private[akka] (
lazy val remoteServerAddress: InetSocketAddress = remoteService.address
lazy val metricsManager: NodeMetricsManager = new LocalNodeMetricsManager(zkClient, Cluster.metricsRefreshInterval).start()
// static nodes
val CLUSTER_PATH = "/" + nodeAddress.clusterName
val MEMBERSHIP_PATH = CLUSTER_PATH + "/members"
@ -317,6 +320,7 @@ class DefaultClusterNode private[akka] (
val ACTOR_UUID_REGISTRY_PATH = CLUSTER_PATH + "/actor-uuid-registry"
val ACTOR_ADDRESS_TO_UUIDS_PATH = CLUSTER_PATH + "/actor-address-to-uuids"
val NODE_TO_ACTOR_UUIDS_PATH = CLUSTER_PATH + "/node-to-actors-uuids"
val NODE_METRICS = CLUSTER_PATH + "/metrics"
val basePaths = List(
CLUSTER_PATH,
@ -327,7 +331,8 @@ class DefaultClusterNode private[akka] (
NODE_TO_ACTOR_UUIDS_PATH,
ACTOR_ADDRESS_TO_UUIDS_PATH,
CONFIGURATION_PATH,
PROVISIONING_PATH)
PROVISIONING_PATH,
NODE_METRICS)
val LEADER_ELECTION_PATH = CLUSTER_PATH + "/leader" // should NOT be part of 'basePaths' only used by 'leaderLock'
@ -343,7 +348,7 @@ class DefaultClusterNode private[akka] (
private val changeListeners = new CopyOnWriteArrayList[ChangeListener]()
// Address -> ClusterActorRef
private val clusterActorRefs = new Index[InetSocketAddress, ClusterActorRef]
private[akka] val clusterActorRefs = new Index[InetSocketAddress, ClusterActorRef]
case class VersionedConnectionState(version: Long, connections: Map[String, Tuple2[InetSocketAddress, ActorRef]])
@ -900,18 +905,7 @@ class DefaultClusterNode private[akka] (
/**
* Creates an ActorRef with a Router to a set of clustered actors.
*/
def ref(actorAddress: String, router: RouterType): ActorRef = {
val inetSocketAddresses = inetSocketAddressesForActor(actorAddress)
EventHandler.debug(this,
"Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]"
.format(actorAddress, router, remoteServerAddress, inetSocketAddresses.map(_._2).mkString("\n\t")))
val actorRef = ClusterActorRef.newRef(router, inetSocketAddresses, actorAddress, Actor.TIMEOUT)
inetSocketAddresses foreach {
case (_, inetSocketAddress) clusterActorRefs.put(inetSocketAddress, actorRef)
}
actorRef.start()
}
def ref(actorAddress: String, router: RouterType): ActorRef = ClusterActorRef.newRef(router, actorAddress, Actor.TIMEOUT)
/**
* Returns the UUIDs of all actors checked out on this node.
@ -1643,7 +1637,11 @@ class MembershipChildListener(self: ClusterNode) extends IZkChildListener with E
// publish NodeConnected and NodeDisconnect events to the listeners
newlyConnectedMembershipNodes foreach (node self.publish(NodeConnected(node)))
newlyDisconnectedMembershipNodes foreach (node self.publish(NodeDisconnected(node)))
newlyDisconnectedMembershipNodes foreach { node
self.publish(NodeDisconnected(node))
// remove metrics of a disconnected node from ZK and local cache
self.metricsManager.removeNodeMetrics(node)
}
}
}
}
@ -1715,8 +1713,6 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
import RemoteClusterDaemon._
import Cluster._
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
override def preRestart(reason: Throwable, msg: Option[Any]) {
EventHandler.debug(this, "RemoteClusterDaemon failed due to [%s] restarting...".format(reason))
}
@ -1865,59 +1861,25 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}
def handle_fun0_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
localActorOf(new Actor() {
self.dispatcher = computeGridDispatcher
def receive = {
case f: Function0[_] try {
f()
} finally {
self.stop()
}
}
}).start ! payloadFor(message, classOf[Function0[Unit]])
actorOf(Props(
self { case f: Function0[_] try { f() } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) ! payloadFor(message, classOf[Function0[Unit]])
}
def handle_fun0_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
localActorOf(new Actor() {
self.dispatcher = computeGridDispatcher
def receive = {
case f: Function0[_] try {
self.reply(f())
} finally {
self.stop()
}
}
}).start forward payloadFor(message, classOf[Function0[Any]])
actorOf(Props(
self { case f: Function0[_] try { self.reply(f()) } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) forward payloadFor(message, classOf[Function0[Any]])
}
def handle_fun1_arg_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
localActorOf(new Actor() {
self.dispatcher = computeGridDispatcher
def receive = {
case (fun: Function[_, _], param: Any) try {
fun.asInstanceOf[Any Unit].apply(param)
} finally {
self.stop()
}
}
}).start ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
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]])
}
def handle_fun1_arg_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
localActorOf(new Actor() {
self.dispatcher = computeGridDispatcher
def receive = {
case (fun: Function[_, _], param: Any) try {
self.reply(fun.asInstanceOf[Any Any](param))
} finally {
self.stop()
}
}
}).start forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
actorOf(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]])
}
def handleFailover(message: ClusterProtocol.RemoteDaemonMessageProtocol) {

View file

@ -7,7 +7,6 @@ import akka.actor._
import akka.util._
import akka.event.EventHandler
import ReflectiveAccess._
import akka.dispatch.Future
import akka.routing._
import RouterType._
import akka.cluster._
@ -15,8 +14,6 @@ import akka.cluster._
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import com.eaio.uuid.UUID
import collection.immutable.Map
import annotation.tailrec
@ -27,19 +24,19 @@ import annotation.tailrec
*/
object ClusterActorRef {
def newRef(
routerType: RouterType,
inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]],
actorAddress: String,
timeout: Long): ClusterActorRef = {
routerType match {
case Direct new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, new DirectRouter())
case Random new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, new RandomRouter())
case RoundRobin new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, new RoundRobinRouter())
def newRef(routerType: RouterType, actorAddress: String, timeout: Long): ClusterActorRef = {
val routerFactory: () Router = routerType match {
case Direct () new DirectRouter
case Random () new RandomRouter()
case RoundRobin () new RoundRobinRouter()
case LeastCPU sys.error("Router LeastCPU not supported yet")
case LeastRAM sys.error("Router LeastRAM not supported yet")
case LeastMessages sys.error("Router LeastMessages not supported yet")
}
val props = RoutedProps.apply().withDeployId(actorAddress).withTimeout(timeout).withRouter(routerFactory)
new ClusterActorRef(props).start()
}
/**
@ -58,48 +55,29 @@ object ClusterActorRef {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class ClusterActorRef private[akka] (
inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]],
_address: String,
_timeout: Long,
val router: Router)
extends UnsupportedActorRef {
private[akka] class ClusterActorRef(props: RoutedProps) extends AbstractRoutedActorRef(props) {
import ClusterActorRef._
ClusterModule.ensureEnabled()
// val address = Address.clusterActorRefPrefix + _address
val address = _address
val addresses = Cluster.node.inetSocketAddressesForActor(address)
EventHandler.debug(this,
"Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]"
.format(address, router, Cluster.node.remoteServerAddress, addresses.map(_._2).mkString("\n\t")))
timeout = _timeout
addresses foreach {
case (_, address) Cluster.node.clusterActorRefs.put(address, this)
}
val connections = new ClusterActorRefConnections((Map[InetSocketAddress, ActorRef]() /: inetSocketAddresses) {
val connections = new ClusterActorRefConnections((Map[InetSocketAddress, ActorRef]() /: addresses) {
case (map, (uuid, inetSocketAddress)) map + (inetSocketAddress -> createRemoteActorRef(address, inetSocketAddress))
})
}, props.connections)
router.init(connections)
def connectionsSize(): Int = connections.size
override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = {
val sender = channel match {
case ref: ActorRef Some(ref)
case _ None
}
router.route(message)(sender)
}
override def postMessageToMailboxAndCreateFutureResultWithTimeout(message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
val sender = channel match {
case ref: ActorRef Some(ref)
case _ None
}
router.route[Any](message, timeout.duration.toMillis)(sender)
}
private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = {
connections.failOver(from, to)
}
@ -131,24 +109,19 @@ class ClusterActorRefConnections() extends RouterConnections {
private val state = new AtomicReference[State]()
def this(connectionMap: Map[InetSocketAddress, ActorRef]) = {
def this(clusteredConnections: Map[InetSocketAddress, ActorRef], explicitConnections: Iterable[ActorRef]) = {
this()
state.set(new State(Long.MinValue, connectionMap))
state.set(new State(Long.MinValue, clusteredConnections, explicitConnections))
}
def version: Long = state.get().version
def connections: Map[InetSocketAddress, ActorRef] = state.get.connections
def versionedIterable = state.get
def versionedIterator = {
val s = state.get
(s.version, s.connections.values)
}
def size: Int = state.get().connections.size
def size(): Int = state.get().iterable.size
def stopAll() {
state.get().connections.values foreach (_.stop()) // shut down all remote connections
state.get().clusteredConnections.values foreach (_.stop()) // shut down all remote connections
}
@tailrec
@ -156,18 +129,18 @@ class ClusterActorRefConnections() extends RouterConnections {
EventHandler.debug(this, "ClusterActorRef failover from [%s] to [%s]".format(from, to))
val oldState = state.get
var change = false
val newMap = oldState.connections map {
var changed = false
val newMap = oldState.clusteredConnections map {
case (`from`, actorRef)
change = true
// actorRef.stop()
changed = true
//actorRef.stop()
(to, createRemoteActorRef(actorRef.address, to))
case other other
}
if (change) {
if (changed) {
//there was a state change, so we are now going to update the state.
val newState = new State(oldState.version + 1, newMap)
val newState = new State(oldState.version + 1, newMap, oldState.explicitConnections)
//if we are not able to update, the state, we are going to try again.
if (!state.compareAndSet(oldState, newState)) failOver(from, to)
@ -180,22 +153,41 @@ class ClusterActorRefConnections() extends RouterConnections {
val oldState = state.get()
//remote the ref from the connections.
var changed = false
//remote the deadRef from the clustered-connections.
var newConnections = Map[InetSocketAddress, ActorRef]()
oldState.connections.keys.foreach(
oldState.clusteredConnections.keys.foreach(
address {
val actorRef: ActorRef = oldState.connections.get(address).get
if (actorRef ne deadRef) newConnections = newConnections + ((address, actorRef))
val actorRef: ActorRef = oldState.clusteredConnections.get(address).get
if (actorRef ne deadRef) {
newConnections = newConnections + ((address, actorRef))
} else {
changed = true
}
})
if (newConnections.size != oldState.connections.size) {
//remove the deadRef also from the explicit connections.
var newExplicitConnections = oldState.explicitConnections.filter(
actorRef
if (actorRef == deadRef) {
changed = true
false
} else {
true
})
if (changed) {
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = new State(oldState.version + 1, newConnections)
val newState = new State(oldState.version + 1, newConnections, newExplicitConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) remove(deadRef)
}
}
case class State(val version: Long, val connections: Map[InetSocketAddress, ActorRef])
class State(version: Long = Integer.MIN_VALUE,
val clusteredConnections: Map[InetSocketAddress, ActorRef],
val explicitConnections: Iterable[ActorRef])
extends VersionedIterable[ActorRef](version, explicitConnections ++ clusteredConnections.values)
}

View file

@ -4,9 +4,10 @@
package akka.cluster
import akka.actor.Actor
import akka.actor.{ Actor, Props }
import Actor._
import akka.cluster._
import akka.dispatch.Dispatchers
import akka.dispatch.{ Dispatchers, Future, PinnedDispatcher }
import akka.util.ListenerManagement
import scala.collection.mutable.{ HashMap, Set }
@ -19,14 +20,13 @@ object FailureDetector {
private case class Register(strategy: FailOverStrategy, address: InetSocketAddress) extends FailureDetectorEvent
private case class Unregister(strategy: FailOverStrategy, address: InetSocketAddress) extends FailureDetectorEvent
private[akka] val registry = Actor.localActorOf[Registry].start()
private[akka] val registry = actorOf(Props(new Registry).copy(dispatcher = new PinnedDispatcher(), localOnly = true))
def register(strategy: FailOverStrategy, address: InetSocketAddress) = registry ! Register(strategy, address)
def unregister(strategy: FailOverStrategy, address: InetSocketAddress) = registry ! Unregister(strategy, address)
private class Registry extends Actor {
self.dispatcher = Dispatchers.newPinnedDispatcher(self)
val strategies = new HashMap[InetSocketAddress, Set[FailOverStrategy]]() {
override def default(k: InetSocketAddress) = Set.empty[FailOverStrategy]

View file

@ -1,6 +0,0 @@
package akka.cluster
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/

View file

@ -0,0 +1,226 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.cluster._
import Cluster._
import akka.cluster.zookeeper._
import akka.actor._
import Actor._
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
import java.util.concurrent.{ ConcurrentHashMap, ConcurrentSkipListSet }
import java.util.concurrent.atomic.AtomicReference
import akka.util.{ Duration, Switch }
import akka.util.Helpers._
import akka.util.duration._
import org.I0Itec.zkclient.exception.ZkNoNodeException
import akka.event.EventHandler
/*
* Instance of the metrics manager running on the node. To keep the fine performance, metrics of all the
* nodes in the cluster are cached internally, and refreshed from monitoring MBeans / Sigar (when if's local node),
* of ZooKeeper (if it's metrics of all the nodes in the cluster) after a specified timeout -
* <code>metricsRefreshTimeout</code>
* <code>metricsRefreshTimeout</code> defaults to 2 seconds, and can be declaratively defined through
* akka.conf:
*
* @exampl {{{
* akka.cluster.metrics-refresh-timeout = 2
* }}}
*/
class LocalNodeMetricsManager(zkClient: AkkaZkClient, private val metricsRefreshTimeout: Duration)
extends NodeMetricsManager {
/*
* Provides metrics of the system that the node is running on, through monitoring MBeans, Hyperic Sigar
* and other systems
*/
lazy private val metricsProvider = SigarMetricsProvider(refreshTimeout.toMillis.toInt) fold ((thrw) {
EventHandler.warning(this, """Hyperic Sigar library failed to load due to %s: %s.
All the metrics will be retreived from monitoring MBeans, and may be incorrect at some platforms.
In order to get better metrics, please put "sigar.jar" to the classpath, and add platform-specific native libary to "java.library.path"."""
.format(thrw.getClass.getName, thrw.getMessage))
new JMXMetricsProvider
},
sigar sigar)
/*
* Metrics of all nodes in the cluster
*/
private val localNodeMetricsCache = new ConcurrentHashMap[String, NodeMetrics]
@volatile
private var _refreshTimeout = metricsRefreshTimeout
/*
* Plugged monitors (both local and cluster-wide)
*/
private val alterationMonitors = new ConcurrentSkipListSet[MetricsAlterationMonitor]
private val _isRunning = new Switch(false)
/*
* If the value is <code>true</code>, metrics manages is started and running. Stopped, otherwise
*/
def isRunning = _isRunning.isOn
/*
* Starts metrics manager. When metrics manager is started, it refreshes cache from ZooKeeper
* after <code>refreshTimeout</code>, and invokes plugged monitors
*/
def start() = {
_isRunning.switchOn { refresh() }
this
}
private[cluster] def metricsForNode(nodeName: String): String = "%s/%s".format(node.NODE_METRICS, nodeName)
/*
* Adds monitor that reacts, when specific conditions are satisfied
*/
def addMonitor(monitor: MetricsAlterationMonitor) = alterationMonitors add monitor
def removeMonitor(monitor: MetricsAlterationMonitor) = alterationMonitors remove monitor
def refreshTimeout_=(newValue: Duration) = _refreshTimeout = newValue
/*
* Timeout after which metrics, cached in the metrics manager, will be refreshed from ZooKeeper
*/
def refreshTimeout = _refreshTimeout
/*
* Stores metrics of the node in ZooKeeper
*/
private[akka] def storeMetricsInZK(metrics: NodeMetrics) = {
val metricsPath = metricsForNode(metrics.nodeName)
if (zkClient.exists(metricsPath)) {
zkClient.writeData(metricsPath, metrics)
} else {
ignore[ZkNoNodeException](zkClient.createEphemeral(metricsPath, metrics))
}
}
/*
* Gets metrics of the node from ZooKeeper
*/
private[akka] def getMetricsFromZK(nodeName: String) = {
zkClient.readData[NodeMetrics](metricsForNode(nodeName))
}
/*
* Removed metrics of the node from local cache and ZooKeeper
*/
def removeNodeMetrics(nodeName: String) = {
val metricsPath = metricsForNode(nodeName)
if (zkClient.exists(metricsPath)) {
ignore[ZkNoNodeException](zkClient.delete(metricsPath))
}
localNodeMetricsCache.remove(nodeName)
}
/*
* Gets metrics of a local node directly from JMX monitoring beans/Hyperic Sigar
*/
def getLocalMetrics = metricsProvider.getLocalMetrics
/*
* Gets metrics of the node, specified by the name. If <code>useCached</code> is true (default value),
* metrics snapshot is taken from the local cache; otherwise, it's retreived from ZooKeeper'
*/
def getMetrics(nodeName: String, useCached: Boolean = true): Option[NodeMetrics] =
if (useCached)
Option(localNodeMetricsCache.get(nodeName))
else
try {
Some(getMetricsFromZK(nodeName))
} catch {
case ex: ZkNoNodeException None
}
/*
* Return metrics of all nodes in the cluster from ZooKeeper
*/
private[akka] def getAllMetricsFromZK: Map[String, NodeMetrics] = {
val metricsPaths = zkClient.getChildren(node.NODE_METRICS).toList.toArray.asInstanceOf[Array[String]]
metricsPaths.flatMap { nodeName getMetrics(nodeName, false).map((nodeName, _)) } toMap
}
/*
* Gets cached metrics of all nodes in the cluster
*/
def getAllMetrics: Array[NodeMetrics] = localNodeMetricsCache.values.asScala.toArray
/*
* Refreshes locally cached metrics from ZooKeeper, and invokes plugged monitors
*/
private[akka] def refresh(): Unit = {
storeMetricsInZK(getLocalMetrics)
refreshMetricsCacheFromZK()
if (isRunning) {
Scheduler.schedule({ () refresh() }, refreshTimeout.length, refreshTimeout.length, refreshTimeout.unit)
invokeMonitors()
}
}
/*
* Refreshes metrics manager cache from ZooKeeper
*/
private def refreshMetricsCacheFromZK(): Unit = {
val allMetricsFromZK = getAllMetricsFromZK
localNodeMetricsCache.keySet.foreach { key
if (!allMetricsFromZK.contains(key))
localNodeMetricsCache.remove(key)
}
// RACY: metrics for the node might have been removed both from ZK and local cache by the moment,
// but will be re-cached, since they're still present in allMetricsFromZK snapshot. Not important, because
// cache will be fixed soon, at the next iteration of refresh
allMetricsFromZK map {
case (node, metrics)
localNodeMetricsCache.put(node, metrics)
}
}
/*
* Invokes monitors with the cached metrics
*/
private def invokeMonitors(): Unit = if (!alterationMonitors.isEmpty) {
// RACY: metrics for some nodes might have been removed/added by that moment. Not important,
// because monitors will be fed with up-to-date metrics shortly, at the next iteration of refresh
val clusterNodesMetrics = getAllMetrics
val localNodeMetrics = clusterNodesMetrics.find(_.nodeName == nodeAddress.nodeName)
val iterator = alterationMonitors.iterator
// RACY: there might be new monitors added after the iterator has been obtained. Not important,
// becuse refresh interval is meant to be very short, and all the new monitors will be called ad the
// next refresh iteration
while (iterator.hasNext) {
val monitor = iterator.next
monitor match {
case localMonitor: LocalMetricsAlterationMonitor
localNodeMetrics.map { metrics
if (localMonitor reactsOn metrics)
localMonitor react metrics
}
case clusterMonitor: ClusterMetricsAlterationMonitor
if (clusterMonitor reactsOn clusterNodesMetrics)
clusterMonitor react clusterNodesMetrics
}
}
}
def stop() = _isRunning.switchOff
}

View file

@ -0,0 +1,154 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.cluster._
import akka.event.EventHandler
import java.lang.management.ManagementFactory
import akka.util.ReflectiveAccess._
import akka.util.Switch
/*
* Snapshot of the JVM / system that's the node is running on
*
* @param nodeName name of the node, where metrics are gathered at
* @param usedHeapMemory amount of heap memory currently used
* @param committedHeapMemory amount of heap memory guaranteed to be available
* @param maxHeapMemory maximum amount of heap memory that can be used
* @param avaiableProcessors number of the processors avalable to the JVM
* @param systemLoadAverage system load average. If OS-specific Sigar's native library is plugged,
* it's used to calculate average load on the CPUs in the system. Otherwise, value is retreived from monitoring
* MBeans. Hyperic Sigar provides more precise values, and, thus, if the library is provided, it's used by default.
*
*/
case class DefaultNodeMetrics(nodeName: String,
usedHeapMemory: Long,
committedHeapMemory: Long,
maxHeapMemory: Long,
avaiableProcessors: Int,
systemLoadAverage: Double) extends NodeMetrics
object MetricsProvider {
/*
* Maximum value of system load average
*/
val MAX_SYS_LOAD_AVG = 1
/*
* Minimum value of system load average
*/
val MIN_SYS_LOAD_AVG = 0
/*
* Default value of system load average
*/
val DEF_SYS_LOAD_AVG = 0.5
}
/*
* Abstracts metrics provider that returns metrics of the system the node is running at
*/
trait MetricsProvider {
/*
* Gets metrics of the local system
*/
def getLocalMetrics: NodeMetrics
}
/*
* Loads JVM metrics through JMX monitoring beans
*/
class JMXMetricsProvider extends MetricsProvider {
import MetricsProvider._
private val memoryMXBean = ManagementFactory.getMemoryMXBean
private val osMXBean = ManagementFactory.getOperatingSystemMXBean
/*
* Validates and calculates system load average
*
* @param avg system load average obtained from a specific monitoring provider (may be incorrect)
* @return system load average, or default value(<code>0.5</code>), if passed value was out of permitted
* bounds (0.0 to 1.0)
*/
@inline
protected final def calcSystemLoadAverage(avg: Double) =
if (avg >= MIN_SYS_LOAD_AVG && avg <= MAX_SYS_LOAD_AVG) avg else DEF_SYS_LOAD_AVG
protected def systemLoadAverage = calcSystemLoadAverage(osMXBean.getSystemLoadAverage)
def getLocalMetrics =
DefaultNodeMetrics(Cluster.nodeAddress.nodeName,
memoryMXBean.getHeapMemoryUsage.getUsed,
memoryMXBean.getHeapMemoryUsage.getCommitted,
memoryMXBean.getHeapMemoryUsage.getMax,
osMXBean.getAvailableProcessors,
systemLoadAverage)
}
/*
* Loads wider range of metrics of a better quality with Hyperic Sigar (native library)
*
* @param refreshTimeout Sigar gathers metrics during this interval
*/
class SigarMetricsProvider private (private val sigarInstance: AnyRef) extends JMXMetricsProvider {
private val reportErrors = new Switch(true)
private val getCpuPercMethod = sigarInstance.getClass.getMethod("getCpuPerc")
private val sigarCpuCombinedMethod = getCpuPercMethod.getReturnType.getMethod("getCombined")
/*
* Wraps reflective calls to Hyperic Sigar
*
* @param f reflective call to Hyperic Sigar
* @param fallback function, which is invoked, if call to Sigar has been finished with exception
*/
private def callSigarMethodOrElse[T](callSigar: T, fallback: T): T =
try callSigar catch {
case thrw
reportErrors.switchOff {
EventHandler.warning(this, "Failed to get metrics from Hyperic Sigar. %s: %s"
.format(thrw.getClass.getName, thrw.getMessage))
}
fallback
}
/*
* Obtains system load average from Sigar
* If the value cannot be obtained, falls back to system load average taken from JMX
*/
override def systemLoadAverage = callSigarMethodOrElse(
calcSystemLoadAverage(sigarCpuCombinedMethod
.invoke(getCpuPercMethod.invoke(sigarInstance)).asInstanceOf[Double]),
super.systemLoadAverage)
}
object SigarMetricsProvider {
/*
* Instantiates Sigar metrics provider through reflections, in order to avoid creating dependencies to
* Hiperic Sigar library
*/
def apply(refreshTimeout: Int): Either[Throwable, MetricsProvider] = try {
for {
sigarInstance createInstance[AnyRef]("org.hyperic.sigar.Sigar", noParams, noArgs).right
sigarProxyCacheClass: Class[_] getClassFor("org.hyperic.sigar.SigarProxyCache").right
} yield new SigarMetricsProvider(sigarProxyCacheClass
.getMethod("newInstance", Array(sigarInstance.getClass, classOf[Int]): _*)
.invoke(null, sigarInstance, new java.lang.Integer(refreshTimeout)))
} catch {
case thrw Left(thrw)
}
}

View file

@ -9,7 +9,7 @@ import akka.actor.{ uuidFrom, newUuid }
import akka.actor._
import DeploymentConfig._
import akka.dispatch.MessageInvocation
import akka.util.ReflectiveAccess
import akka.util.{ ReflectiveAccess, Duration }
import akka.cluster.{ RemoteClientSettings, MessageSerializer }
import akka.cluster.protocol.RemoteProtocol
import RemoteProtocol._
@ -22,6 +22,7 @@ import com.google.protobuf.ByteString
import com.eaio.uuid.UUID
import akka.event.EventHandler
import java.util.{ LinkedList, Collections }
/**
* Module for local actor serialization.
@ -60,6 +61,11 @@ object ActorSerialization {
serializeMailBox: Boolean,
replicationScheme: ReplicationScheme): SerializedActorRefProtocol = {
val localRef: Option[LocalActorRef] = actorRef match {
case l: LocalActorRef Some(l)
case _ None
}
val lifeCycleProtocol: Option[LifeCycleProtocol] = {
actorRef.lifeCycle match {
case Permanent Some(LifeCycleProtocol.newBuilder.setLifeCycle(LifeCycleType.PERMANENT).build)
@ -71,9 +77,11 @@ object ActorSerialization {
val builder = SerializedActorRefProtocol.newBuilder
.setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build)
.setAddress(actorRef.address)
.setActorClassname(actorRef.actorInstance.get.getClass.getName)
.setTimeout(actorRef.timeout)
if (localRef.isDefined)
builder.setActorClassname(localRef.get.actorInstance.get.getClass.getName) //TODO FIXME Why is the classname needed anymore?
replicationScheme match {
case _: Transient | Transient
builder.setReplicationStorage(ReplicationStorageType.TRANSIENT)
@ -86,47 +94,53 @@ object ActorSerialization {
builder.setReplicationStorage(storageType)
val strategyType = strategy match {
case _: WriteBehind | WriteBehind ReplicationStrategyType.WRITE_BEHIND
case _: WriteThrough | WriteThrough ReplicationStrategyType.WRITE_THROUGH
case _: WriteBehind ReplicationStrategyType.WRITE_BEHIND
case _: WriteThrough ReplicationStrategyType.WRITE_THROUGH
}
builder.setReplicationStrategy(strategyType)
}
if (serializeMailBox == true) {
if (actorRef.mailbox eq null) throw new IllegalActorStateException("Can't serialize an actor that has not been started.")
val messages =
actorRef.mailbox match {
lifeCycleProtocol.foreach(builder.setLifeCycle(_))
actorRef.supervisor.foreach(s builder.setSupervisor(RemoteActorSerialization.toRemoteActorRefProtocol(s)))
localRef foreach { l
if (serializeMailBox) {
l.mailbox match {
case null throw new IllegalActorStateException("Can't serialize an actor that has not been started.")
case q: java.util.Queue[_]
val l = new scala.collection.mutable.ListBuffer[MessageInvocation]
val it = q.iterator
while (it.hasNext) l += it.next.asInstanceOf[MessageInvocation]
l
l map { m
RemoteActorSerialization.createRemoteMessageProtocolBuilder(
Option(m.receiver),
Left(actorRef.uuid),
actorRef.address,
actorRef.timeout,
Right(m.message),
false,
m.channel match {
case a: ActorRef Some(a)
case _ None
})
} foreach {
builder.addMessages(_)
}
}
}
val requestProtocols =
messages.map(m
RemoteActorSerialization.createRemoteMessageProtocolBuilder(
Some(actorRef),
Left(actorRef.uuid),
actorRef.address,
actorRef.timeout,
Right(m.message),
false,
actorRef.getSender))
requestProtocols.foreach(builder.addMessages(_))
l.receiveTimeout.foreach(builder.setReceiveTimeout(_))
val actorInstance = l.actorInstance.get
Serialization.serialize(actorInstance.asInstanceOf[T]) match {
case Right(bytes) builder.setActorInstance(ByteString.copyFrom(bytes))
case Left(exception) throw new Exception("Error serializing : " + actorInstance.getClass.getName)
}
val stack = l.hotswap
if (!stack.isEmpty)
builder.setHotswapStack(ByteString.copyFrom(akka.serialization.JavaSerializer.toBinary(stack)))
}
actorRef.receiveTimeout.foreach(builder.setReceiveTimeout(_))
Serialization.serialize(actorRef.actor.asInstanceOf[T]) match {
case Right(bytes) builder.setActorInstance(ByteString.copyFrom(bytes))
case Left(exception) throw new Exception("Error serializing : " + actorRef.actor.getClass.getName)
}
lifeCycleProtocol.foreach(builder.setLifeCycle(_))
actorRef.supervisor.foreach(s builder.setSupervisor(RemoteActorSerialization.toRemoteActorRefProtocol(s)))
// if (!actorRef.hotswap.isEmpty) builder.setHotswapStack(ByteString.copyFrom(Serializers.Java.toBinary(actorRef.hotswap)))
if (!actorRef.hotswap.isEmpty) builder.setHotswapStack(ByteString.copyFrom(akka.serialization.JavaSerializer.toBinary(actorRef.hotswap)))
builder.build
}
@ -145,19 +159,6 @@ object ActorSerialization {
EventHandler.debug(this, "Deserializing SerializedActorRefProtocol to LocalActorRef:\n%s".format(protocol))
val lifeCycle =
if (protocol.hasLifeCycle) {
protocol.getLifeCycle.getLifeCycle match {
case LifeCycleType.PERMANENT Permanent
case LifeCycleType.TEMPORARY Temporary
case unknown throw new IllegalActorStateException("LifeCycle type is not valid [" + unknown + "]")
}
} else UndefinedLifeCycle
val supervisor =
if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
else None
// import ReplicationStorageType._
// import ReplicationStrategyType._
// val replicationScheme =
@ -180,7 +181,7 @@ object ActorSerialization {
// }
// } else Transient
val hotswap =
val storedHotswap =
try {
Serialization.deserialize(
protocol.getHotswapStack.toByteArray,
@ -193,17 +194,26 @@ object ActorSerialization {
case e: Exception Stack[PartialFunction[Any, Unit]]()
}
val classLoader = loader.getOrElse(this.getClass.getClassLoader)
val factory = () {
val actorClass = classLoader.loadClass(protocol.getActorClassname)
try {
Serialization.deserialize(protocol.getActorInstance.toByteArray, actorClass, loader) match {
case Right(r) r.asInstanceOf[Actor]
case Left(ex) throw new Exception("Cannot de-serialize : " + actorClass)
val storedLifeCycle =
if (protocol.hasLifeCycle) {
protocol.getLifeCycle.getLifeCycle match {
case LifeCycleType.PERMANENT Permanent
case LifeCycleType.TEMPORARY Temporary
case unknown UndefinedLifeCycle
}
} catch {
case e: Exception actorClass.newInstance.asInstanceOf[Actor]
} else UndefinedLifeCycle
val storedSupervisor =
if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
else None
val classLoader = loader.getOrElse(this.getClass.getClassLoader)
val bytes = protocol.getActorInstance.toByteArray
val actorClass = classLoader.loadClass(protocol.getActorClassname)
val factory = () {
Serialization.deserialize(bytes, actorClass, loader) match {
case Right(r) r.asInstanceOf[Actor]
case Left(ex) throw new Exception("Cannot de-serialize : " + actorClass)
}
}
@ -212,18 +222,25 @@ object ActorSerialization {
case None uuidFrom(protocol.getUuid.getHigh, protocol.getUuid.getLow)
}
val ar = new LocalActorRef(
actorUuid,
protocol.getAddress,
if (protocol.hasTimeout) protocol.getTimeout else Actor.TIMEOUT,
if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None,
lifeCycle,
supervisor,
hotswap,
factory)
val props = Props(creator = factory,
timeout = if (protocol.hasTimeout) protocol.getTimeout else Timeout.default,
lifeCycle = storedLifeCycle,
supervisor = storedSupervisor //TODO what dispatcher should it use?
//TODO what faultHandler should it use?
//
)
val receiveTimeout = if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None //TODO FIXME, I'm expensive and slow
val ar = new LocalActorRef(actorUuid, protocol.getAddress, props, receiveTimeout, storedHotswap)
//Deserialize messages
{
val iterator = protocol.getMessagesList.iterator()
while (iterator.hasNext())
ar ! MessageSerializer.deserialize(iterator.next().getMessage, Some(classLoader)) //TODO This is broken, why aren't we preserving the sender?
}
val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]]
messages.foreach(message ar ! MessageSerializer.deserialize(message.getMessage, Some(classLoader)))
ar
}
}

View file

@ -0,0 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.cluster.metrics-refresh-timeout = 1

View file

@ -0,0 +1 @@
-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991

View file

@ -0,0 +1,134 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics.local
import akka.cluster._
import akka.actor._
import Actor._
import Cluster._
import akka.dispatch._
import akka.util.Duration
import akka.util.duration._
import akka.cluster.metrics._
import java.util.concurrent.atomic.AtomicInteger
object LocalMetricsMultiJvmSpec {
val NrOfNodes = 1
}
class LocalMetricsMultiJvmNode1 extends MasterClusterTestNode {
import LocalMetricsMultiJvmSpec._
val testNodes = NrOfNodes
override def beforeAll = {
super.beforeAll()
node
}
override def afterAll = {
node.shutdown()
super.afterAll()
}
"Metrics manager" must {
def timeout = node.metricsManager.refreshTimeout
"be initialized with refresh timeout value, specified in akka.conf" in {
timeout must be(1.second)
}
"return up-to-date local node metrics straight from MBeans/Sigar" in {
node.metricsManager.getLocalMetrics must not be (null)
node.metricsManager.getLocalMetrics.systemLoadAverage must be(0.5 plusOrMinus 0.5)
}
"return metrics cached in the MetricsManagerLocalMetrics" in {
node.metricsManager.getMetrics(nodeAddress.nodeName) must not be (null)
}
"return local node metrics from ZNode" in {
node.metricsManager.getMetrics(nodeAddress.nodeName, false) must not be (null)
}
"return cached metrics of all nodes in the cluster" in {
node.metricsManager.getAllMetrics.size must be(1)
node.metricsManager.getAllMetrics.find(_.nodeName == "node1") must not be (null)
}
"throw no exceptions, when user attempts to get metrics of a non-existing node" in {
node.metricsManager.getMetrics("nonexisting") must be(None)
node.metricsManager.getMetrics("nonexisting", false) must be(None)
}
"regularly update cached metrics" in {
val oldMetrics = node.metricsManager.getLocalMetrics
Thread sleep timeout.toMillis
node.metricsManager.getLocalMetrics must not be (oldMetrics)
}
"allow to track JVM state and bind handles through MetricsAlterationMonitors" in {
val monitorReponse = new DefaultPromise[String]
node.metricsManager.addMonitor(new LocalMetricsAlterationMonitor {
val id = "heapMemoryThresholdMonitor"
def reactsOn(metrics: NodeMetrics) = metrics.usedHeapMemory > 1
def react(metrics: NodeMetrics) = monitorReponse.completeWithResult("Too much memory is used!")
})
monitorReponse.get must be("Too much memory is used!")
}
class FooMonitor(monitorWorked: AtomicInteger) extends LocalMetricsAlterationMonitor {
val id = "fooMonitor"
def reactsOn(metrics: NodeMetrics) = true
def react(metrics: NodeMetrics) = monitorWorked.set(monitorWorked.get + 1)
}
"allow to unregister the monitor" in {
val monitorWorked = new AtomicInteger(0)
val fooMonitor = new FooMonitor(monitorWorked)
node.metricsManager.addMonitor(fooMonitor)
node.metricsManager.removeMonitor(fooMonitor)
val oldValue = monitorWorked.get
Thread sleep timeout.toMillis
monitorWorked.get must be(oldValue)
}
"stop notifying monitors, when stopped" in {
node.metricsManager.stop()
val monitorWorked = new AtomicInteger(0)
node.metricsManager.addMonitor(new LocalMetricsAlterationMonitor {
val id = "fooMonitor"
def reactsOn(metrics: NodeMetrics) = true
def react(metrics: NodeMetrics) = monitorWorked.set(monitorWorked.get + 1)
})
monitorWorked.get must be(0)
node.metricsManager.start()
Thread sleep (timeout.toMillis * 2)
monitorWorked.get must be > (1)
}
}
}

View file

@ -0,0 +1,3 @@
akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"

View file

@ -0,0 +1 @@
-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991

View file

@ -0,0 +1,3 @@
akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"

View file

@ -0,0 +1 @@
-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992

View file

@ -0,0 +1,133 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics.remote
import akka.cluster._
import akka.actor._
import Actor._
import Cluster._
import akka.dispatch._
import akka.util.Duration
import akka.util.duration._
import akka.cluster.metrics._
import java.util.concurrent._
import atomic.AtomicInteger
object RemoteMetricsMultiJvmSpec {
val NrOfNodes = 2
val MetricsRefreshTimeout = 100.millis
}
class AllMetricsAvailableMonitor(_id: String, completionLatch: CountDownLatch, clusterSize: Int) extends ClusterMetricsAlterationMonitor {
val id = _id
def reactsOn(allMetrics: Array[NodeMetrics]) = allMetrics.size == clusterSize
def react(allMetrics: Array[NodeMetrics]) = completionLatch.countDown
}
class RemoteMetricsMultiJvmNode1 extends MasterClusterTestNode {
import RemoteMetricsMultiJvmSpec._
val testNodes = NrOfNodes
"Metrics manager" must {
"provide metrics of all nodes in the cluster" in {
val allMetricsAvaiable = new CountDownLatch(1)
node.metricsManager.refreshTimeout = MetricsRefreshTimeout
node.metricsManager.addMonitor(new AllMetricsAvailableMonitor("all-metrics-available", allMetricsAvaiable, NrOfNodes))
LocalCluster.barrier("node-start", NrOfNodes).await()
allMetricsAvaiable.await()
LocalCluster.barrier("check-all-remote-metrics", NrOfNodes) {
node.metricsManager.getAllMetrics.size must be(2)
}
val cachedMetrics = node.metricsManager.getMetrics("node2")
val metricsFromZnode = node.metricsManager.getMetrics("node2", false)
LocalCluster.barrier("check-single-remote-metrics", NrOfNodes) {
cachedMetrics must not be (null)
metricsFromZnode must not be (null)
}
Thread sleep MetricsRefreshTimeout.toMillis
LocalCluster.barrier("remote-metrics-is-updated", NrOfNodes) {
node.metricsManager.getMetrics("node2") must not be (cachedMetrics)
node.metricsManager.getMetrics("node2", false) must not be (metricsFromZnode)
}
val someMetricsGone = new CountDownLatch(1)
node.metricsManager.addMonitor(new AllMetricsAvailableMonitor("some-metrics-gone", someMetricsGone, 1))
LocalCluster.barrier("some-nodes-leave", NrOfNodes).await()
someMetricsGone.await(10, TimeUnit.SECONDS) must be(true)
node.metricsManager.getMetrics("node2") must be(None)
node.metricsManager.getMetrics("node2", false) must be(None)
node.metricsManager.getAllMetrics.size must be(1)
node.shutdown()
}
}
}
class RemoteMetricsMultiJvmNode2 extends ClusterTestNode {
import RemoteMetricsMultiJvmSpec._
val testNodes = NrOfNodes
"Metrics manager" must {
"provide metrics of all nodes in the cluster" in {
val allMetricsAvaiable = new CountDownLatch(1)
node.metricsManager.refreshTimeout = MetricsRefreshTimeout
node.metricsManager.addMonitor(new AllMetricsAvailableMonitor("all-metrics-available", allMetricsAvaiable, NrOfNodes))
LocalCluster.barrier("node-start", NrOfNodes).await()
allMetricsAvaiable.await()
LocalCluster.barrier("check-all-remote-metrics", NrOfNodes) {
node.metricsManager.getAllMetrics.size must be(2)
}
val cachedMetrics = node.metricsManager.getMetrics("node1")
val metricsFromZnode = node.metricsManager.getMetrics("node1", false)
LocalCluster.barrier("check-single-remote-metrics", NrOfNodes) {
cachedMetrics must not be (null)
metricsFromZnode must not be (null)
}
Thread sleep MetricsRefreshTimeout.toMillis
LocalCluster.barrier("remote-metrics-is-updated", NrOfNodes) {
node.metricsManager.getMetrics("node1") must not be (cachedMetrics)
node.metricsManager.getMetrics("node1", false) must not be (metricsFromZnode)
}
LocalCluster.barrier("some-nodes-leave", NrOfNodes) {
node.shutdown()
}
}
}
}

View file

@ -21,13 +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 = localActorOf[MyJavaSerializableActor].start()
val actor1 = localActorOf[MyJavaSerializableActor].start().asInstanceOf[LocalActorRef]
(actor1 ? "hello").get should equal("world 1")
(actor1 ? "hello").get should equal("world 2")
val bytes = toBinary(actor1)
val actor2 = fromBinary(bytes)
actor2.start()
val actor2 = fromBinary(bytes).start().asInstanceOf[LocalActorRef]
(actor2 ? "hello").get should equal("world 3")
actor2.receiveTimeout should equal(Some(1000))
@ -37,7 +36,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
it("should be able to serialize and deserialize a MyStatelessActorWithMessagesInMailbox") {
val actor1 = localActorOf[MyStatelessActorWithMessagesInMailbox].start()
val actor1 = localActorOf[MyStatelessActorWithMessagesInMailbox].start().asInstanceOf[LocalActorRef]
(actor1 ! "hello")
(actor1 ! "hello")
(actor1 ! "hello")
@ -49,12 +48,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
(actor1 ! "hello")
(actor1 ! "hello")
actor1.getDispatcher.mailboxSize(actor1) should be > (0)
val actor2 = fromBinary(toBinary(actor1))
val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
actor2.getDispatcher.mailboxSize(actor1) should be > (0)
(actor2 ? "hello-reply").get should equal("world")
val actor3 = fromBinary(toBinary(actor1, false))
val actor3 = fromBinary(toBinary(actor1, false)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
actor3.getDispatcher.mailboxSize(actor1) should equal(0)
(actor3 ? "hello-reply").get should equal("world")
@ -63,7 +62,7 @@ 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 = localActorOf[PersonActorWithMessagesInMailbox].start()
val actor1 = localActorOf[PersonActorWithMessagesInMailbox].start().asInstanceOf[LocalActorRef]
(actor1 ! p1)
(actor1 ! p1)
(actor1 ! p1)
@ -75,12 +74,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
(actor1 ! p1)
(actor1 ! p1)
actor1.getDispatcher.mailboxSize(actor1) should be > (0)
val actor2 = fromBinary(toBinary(actor1))
val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
actor2.getDispatcher.mailboxSize(actor1) should be > (0)
(actor2 ? "hello-reply").get should equal("hello")
val actor3 = fromBinary(toBinary(actor1, false))
val actor3 = fromBinary(toBinary(actor1, false)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
actor3.getDispatcher.mailboxSize(actor1) should equal(0)
(actor3 ? "hello-reply").get should equal("hello")
@ -107,7 +106,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
describe("serialize actor that accepts protobuf message") {
it("should serialize") {
val actor1 = localActorOf[MyActorWithProtobufMessagesInMailbox].start()
val actor1 = localActorOf[MyActorWithProtobufMessagesInMailbox].start().asInstanceOf[LocalActorRef]
val msg = MyMessage(123, "debasish ghosh", true)
val b = ProtobufProtocol.MyMessage.newBuilder.setId(msg.id).setName(msg.name).setStatus(msg.status).build
(actor1 ! b)
@ -121,12 +120,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll
(actor1 ! b)
(actor1 ! b)
actor1.getDispatcher.mailboxSize(actor1) should be > (0)
val actor2 = fromBinary(toBinary(actor1))
val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
actor2.getDispatcher.mailboxSize(actor1) should be > (0)
(actor2 ? "hello-reply").get should equal("world")
val actor3 = fromBinary(toBinary(actor1, false))
val actor3 = fromBinary(toBinary(actor1, false)).asInstanceOf[LocalActorRef]
Thread.sleep(1000)
actor3.getDispatcher.mailboxSize(actor1) should equal(0)
(actor3 ? "hello-reply").get should equal("world")

View file

@ -30,6 +30,7 @@ The durable mailboxes currently supported are:
- ``ZooKeeperDurableMailboxStorage`` -- backed by ZooKeeper
- ``BeanstalkDurableMailboxStorage`` -- backed by Beanstalkd
- ``MongoNaiveDurableMailboxStorage`` -- backed by MongoDB
We'll walk through each one of these in detail in the sections below.
Soon Akka will also have:
@ -235,7 +236,7 @@ MongoDB is a fast, lightweight and scalable document-oriented database. It cont
features cohesive to a fast, reliable & durable queueing mechanism which the Akka Mailbox takes advantage of.
Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver (often known as `Hammersmith <http://github.com/bwmcadams/hammersmith>`_ and ``com.mongodb.async``) and as such are purely callback based with a Netty network layer. This makes them extremely fast & lightweight versus building on other MongoDB implementations such as `mongo-java-driver <http://github.com/mongodb/mongo-java-driver>`_ and `Casbah <http://github.com/mongodb/casbah`_.
Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver (often known as `Hammersmith <http://github.com/bwmcadams/hammersmith>`_ and ``com.mongodb.async``) and as such are purely callback based with a Netty network layer. This makes them extremely fast & lightweight versus building on other MongoDB implementations such as `mongo-java-driver <http://github.com/mongodb/mongo-java-driver>`_ and `Casbah <http://github.com/mongodb/casbah>`_.
You will need to configure the URI for the MongoDB server, using the URI Format specified in the `MongoDB Documentation <http://www.mongodb.org/display/DOCS/Connections>`_. This is done in
the ``akka.actor.mailbox.mongodb`` section in the ``akka.conf`` configuration

View file

@ -1,9 +1,9 @@
//#imports
package akka.tutorial.scala.first
import _root_.akka.routing.{RoutedProps, Routing, CyclicIterator}
import akka.actor.{Actor, PoisonPill}
import Actor._
import akka.routing.{Routing, CyclicIterator}
import Routing._
import System.{currentTimeMillis => now}
@ -65,7 +65,14 @@ object Pi extends App {
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker].start())
// wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start()
val router = Routing.actorOf(
RoutedProps.apply
.withRoundRobinRouter
.withConnections(workers)
.withDeployId("pi")
)
loadBalancerActor(CyclicIterator(workers)).start()
//#create-workers
//#master-receive

View file

@ -9,31 +9,38 @@ The Java Memory Model
---------------------
Prior to Java 5, the Java Memory Model (JMM) was ill defined. It was possible to get all kinds of strange results when
shared memory was accessed by multiple threads, such as:
* a thread not seeing values written by other threads: a visibility problem
* a thread observing 'impossible' behavior of other threads, caused by instructions not being executed in the order
expected: an instruction reordering problem.
expected: an instruction reordering problem.
With the implementation of JSR 133 in Java 5, a lot of these issues have been resolved. The JMM is a set of rules based
on the "happens-before" relation, which constrain when one memory access must happen before another, and conversely,
when they are allowed to happen out of order. Two examples of these rules are:
* **The monitor lock rule:** a release of a lock happens before every subsequent acquire of the same lock.
* **The volatile variable rule:** a write of a volatile variable happens before every subsequent read of the same volatile variable
Although the JMM can seem complicated, the specification tries to find a balance between ease of use and the ability to
write performant and scalable concurrent data structures.
Actors and the Java Memory Model
--------------------------------
With the Actors implementation in Akka, there are two ways multiple threads can execute actions on shared memory:
* if a message is sent to an actor (e.g. by another actor). In most cases messages are immutable, but if that message
is not a properly constructed immutable object, without a "happens before" rule, it would be possible for the receiver
to see partially initialized data structures and possibly even values out of thin air (longs/doubles).
is not a properly constructed immutable object, without a "happens before" rule, it would be possible for the receiver
to see partially initialized data structures and possibly even values out of thin air (longs/doubles).
* if an actor makes changes to its internal state while processing a message, and accesses that state while processing
another message moments later. It is important to realize that with the actor model you don't get any guarantee that
the same thread will be executing the same actor for different messages.
another message moments later. It is important to realize that with the actor model you don't get any guarantee that
the same thread will be executing the same actor for different messages.
To prevent visibility and reordering problems on actors, Akka guarantees the following two "happens before" rules:
* **The actor send rule:** the send of the message to an actor happens before the receive of that message by the same actor.
* **The actor subsequent processing rule:** processing of one message happens before processing of the next message by the same actor.
Both rules only apply for the same actor instance and are not valid if different actors are used.
STM and the Java Memory Model

View file

@ -159,7 +159,9 @@ Here is the layout that Maven created::
As you can see we already have a Java source file called ``App.java``, let's now rename it to ``Pi.java``.
We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. It should now look something like this:
We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. The Akka Maven repository can be found at `<http://akka.io/repository>`_
and Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
It should now look something like this:
.. code-block:: xml
@ -186,9 +188,9 @@ We also need to edit the ``pom.xml`` build file. Let's add the dependency we nee
<repositories>
<repository>
<id>Akka</id>
<name>Akka Maven2 Repository</name>
<url>http://akka.io/repository/</url>
<id>typesafe</id>
<name>Typesafe Repository</name>
<url>http://repo.typesafe.com/typesafe/releases/</url>
</repository>
</repositories>

View file

@ -158,44 +158,21 @@ If you have not already done so, now is the time to create an Eclipse project fo
Using SBT in Eclipse
^^^^^^^^^^^^^^^^^^^^
If you are an `SBT <http://code.google.com/p/simple-build-tool/>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbt-eclipse`` plugin. This adds support for generating Eclipse project files from your SBT project. You need to update your SBT plugins definition in ``project/plugins``::
import sbt._
class TutorialPlugins(info: ProjectInfo) extends PluginDefinition(info) {
// eclipsify plugin
lazy val eclipse = "de.element34" % "sbt-eclipsify" % "0.7.0"
val akkaRepo = "Akka Repo" at "http://akka.io/repository"
val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT"
}
and then update your SBT project definition by mixing in ``Eclipsify`` in your project definition::
import sbt._
import de.element34.sbteclipsify._
class MySbtProject(info: ProjectInfo) extends DefaultProject(info)
with Eclipsify with AkkaProject {
// the project definition here
// akka dependencies
}
If you are an `SBT <https://github.com/harrah/xsbt/wiki>`_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbteclipse`` plugin. This adds support for generating Eclipse project files from your SBT project.
You need to install the plugin as described in the `README of sbteclipse <https://github.com/typesafehub/sbteclipse>`_
Then run the ``eclipse`` target to generate the Eclipse project::
dragos@dragos-imac pi $ sbt eclipse
[info] Building project AkkaPi 1.0 against Scala 2.9.0
[info] using MySbtProject with sbt 0.7.4 and Scala 2.7.7
[info]
[info] == eclipse ==
[info] Creating eclipse project...
[info] == eclipse ==
[success] Successful.
[info]
[info] Total time: 0 s, completed Apr 20, 2011 2:48:03 PM
[info]
[info] Total session time: 1 s, completed Apr 20, 2011 2:48:03 PM
[success] Build completed successfully.
$ sbt
> eclipse
The options `create-src` and `with-sources` are useful::
$ sbt
> eclipse create-src with-sources
* create-src to create the common source directories, e.g. src/main/scala, src/main/test
* with-sources to create source attachments for the library dependencies
Next you need to import this project in Eclipse, by choosing ``Eclipse/Import.. Existing Projects into Workspace``. Navigate to the directory where you defined your SBT project and choose import:

View file

@ -146,59 +146,38 @@ Downloading and installing SBT
SBT, short for 'Simple Build Tool' is an excellent build system written in Scala. It uses Scala to write the build scripts which gives you a lot of power. It has a plugin architecture with many plugins available, something that we will take advantage of soon. SBT is the preferred way of building software in Scala and is probably the easiest way of getting through this tutorial. If you want to use SBT for this tutorial then follow the following instructions, if not you can skip this section and the next.
First browse to `http://code.google.com/p/simple-build-tool/downloads/list <http://code.google.com/p/simple-build-tool/downloads/list>`_ and download the ``0.7.6.RC0`` distribution.
To install SBT and create a project for this tutorial it is easiest to follow the instructions on `http://code.google.com/p/simple-build-tool/wiki/Setup <http://code.google.com/p/simple-build-tool/wiki/Setup>`_.
To install SBT and create a project for this tutorial it is easiest to follow the instructions on `https://github.com/harrah/xsbt/wiki/Setup <https://github.com/harrah/xsbt/wiki/Setup>`_.
Now we need to create our first Akka project. You could add the dependencies manually to the build script, but the easiest way is to use Akka's SBT Plugin, covered in the next section.
Creating an Akka SBT project
----------------------------
If you have not already done so, now is the time to create an SBT project for our tutorial. You do that by stepping into the directory you want to create your project in and invoking the ``sbt`` command answering the questions for setting up your project (just pressing ENTER will choose the default in square brackets)::
If you have not already done so, now is the time to create an SBT project for our tutorial. You do that by adding the following content to ``build.sbt`` file in the directory you want to create your project in::
$ sbt
Project does not exist, create new project? (y/N/s) y
Name: Tutorial 1
Organization: Hakkers Inc
Version [1.0]:
Scala version [2.9.0]:
sbt version [0.7.6.RC0]:
name := "My Project"
Now we have the basis for an SBT project. Akka has an SBT Plugin making it very easy to use Akka is an SBT-based project so let's use that.
version := "1.0"
To use the plugin, first add a plugin definition to your SBT project by creating a ``Plugins.scala`` file in the ``project/plugins`` directory containing::
scalaVersion := "2.9.0-1"
import sbt._
resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/"
class Plugins(info: ProjectInfo) extends PluginDefinition(info) {
val akkaRepo = "Akka Repo" at "http://akka.io/repository"
val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT"
}
libraryDependencies += "se.scalablesolutions.akka" % "akka-actor" % "1.2-SNAPSHOT"
Now we need to create a project definition using our Akka SBT plugin. We do that by creating a ``project/build/Project.scala`` file containing::
Create a directory ``src/main/scala`` in which you will store the Scala source files.
import sbt._
Not needed in this tutorial, but if you would like to use additional Akka modules beyond ``akka-actor``, you can add these as ``libraryDependencies`` in ``build.sbt``. Note that there must be a blank line between each. Here is an example adding ``akka-remote`` and ``akka-stm``::
class TutorialOneProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject
libraryDependencies += "se.scalablesolutions.akka" % "akka-actor" % "1.2-SNAPSHOT"
libraryDependencies += "se.scalablesolutions.akka" % "akka-remote" % "1.2-SNAPSHOT"
libraryDependencies += "se.scalablesolutions.akka" % "akka-stm" % "1.2-SNAPSHOT"
The magic is in mixing in the ``AkkaProject`` trait.
So, now we are all set.
Not needed in this tutorial, but if you would like to use additional Akka modules beyond ``akka-actor``, you can add these as "module configurations" in the project file. Here is an example adding ``akka-remote`` and ``akka-stm``::
class AkkaSampleProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject {
val akkaSTM = akkaModule("stm")
val akkaRemote = akkaModule("remote")
}
So, now we are all set. Just one final thing to do; make SBT download the dependencies it needs. That is done by invoking::
> reload
> update
The first reload command is needed because we have changed the project definition since the sbt session started.
SBT itself needs a whole bunch of dependencies but our project will only need one; ``akka-actor-2.0-SNAPSHOT.jar``. SBT downloads that as well.
SBT itself needs a whole bunch of dependencies but our project will only need one; ``akka-actor-2.0-SNAPSHOT.jar``. SBT will download that as well.
Start writing the code
----------------------
@ -537,8 +516,6 @@ Run it inside SBT
If you used SBT, then you can run the application directly inside SBT. First you need to compile the project::
$ sbt
> update
...
> compile
...

View file

@ -75,7 +75,8 @@ More information is available in the documentation of the :ref:`microkernel`.
Using a build tool
------------------
Akka can be used with build tools that support Maven repositories. The Akka Maven repository can be found at `<http://akka.io/repository>`_.
Akka can be used with build tools that support Maven repositories. The Akka Maven repository can be found at `<http://akka.io/repository>`_
and Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
Using Akka with Maven
---------------------
@ -90,9 +91,9 @@ Summary of the essential parts for using Akka with Maven:
.. code-block:: xml
<repository>
<id>Akka</id>
<name>Akka Maven2 Repository</name>
<url>http://akka.io/repository/ </url>
<id>typesafe</id>
<name>Typesafe Repository</name>
<url>http://repo.typesafe.com/typesafe/releases/</url>
</repository>
2) Add the Akka dependencies. For example, here is the dependency for Akka Actor 2.0-SNAPSHOT:
@ -115,44 +116,19 @@ can be found in the :ref:`getting-started-first-scala`.
Summary of the essential parts for using Akka with SBT:
1) Akka has an SBT plugin which makes it very easy to get started with Akka and SBT.
SBT installation instructions on `https://github.com/harrah/xsbt/wiki/Setup <https://github.com/harrah/xsbt/wiki/Setup>`_
The Scala version in your SBT project needs to match the version that Akka is built against. For Akka 2.0-SNAPSHOT this is
Scala version 2.9.0.
``build.sbt`` file::
To use the plugin, first add a plugin definition to your SBT project by creating project/plugins/Plugins.scala with:
name := "My Project"
.. code-block:: scala
version := "1.0"
import sbt._
scalaVersion := "2.9.0-1"
class Plugins(info: ProjectInfo) extends PluginDefinition(info) {
val akkaRepo = "Akka Repo" at "http://akka.io/repository"
val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT"
}
resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/"
*Note: the plugin version matches the Akka version provided. The current release is 2.0-SNAPSHOT.*
2) Then mix the AkkaProject trait into your project definition. For example:
.. code-block:: scala
class MyProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject
*Note: This adds akka-actor as a dependency by default.*
If you also want to include other Akka modules there is a convenience method: ``akkaModule``. For example, you can add extra Akka modules by adding any of the following lines to your project class:
.. code-block:: scala
val akkaStm = akkaModule("stm")
val akkaTypedActor = akkaModule("typed-actor")
val akkaRemote = akkaModule("remote")
val akkaHttp = akkaModule("http")
val akkaAmqp = akkaModule("amqp")
val akkaCamel = akkaModule("camel")
val akkaCamelTyped = akkaModule("camel-typed")
val akkaSpring = akkaModule("spring")
libraryDependencies += "se.scalablesolutions.akka" % "akka-actor" % "2.0-SNAPSHOT"
Using Akka with Eclipse
@ -161,6 +137,8 @@ Using Akka with Eclipse
Information about how to use Akka with Eclipse, including how to create an Akka Eclipse project from scratch,
can be found in the :ref:`getting-started-first-scala-eclipse`.
Setup SBT project and then use `sbteclipse <https://github.com/typesafehub/sbteclipse>`_ to generate Eclipse project.
Using Akka with IntelliJ IDEA
-----------------------------

View file

@ -233,16 +233,13 @@ Here is the API and how to use it from within an 'Actor':
getContext().unlink(actorRef);
// starts and links Actors atomically
getContext().startLink(actorRef);
getContext().startLinkRemote(actorRef);
getContext().link(actorRef).start();
// spawns (creates and starts) actors
getContext().spawn(MyActor.class);
getContext().spawnRemote(MyActor.class);
// spawns and links Actors atomically
getContext().spawnLink(MyActor.class);
getContext().spawnLinkRemote(MyActor.class);
A child actor can tell the supervising actor to unlink him by sending him the 'Unlink(this)' message. When the supervisor receives the message he will unlink and shut down the child. The supervisor for an actor is available in the 'supervisor: Option[Actor]' method in the 'ActorRef' class. Here is how it can be used.

View file

@ -31,6 +31,20 @@ Akka uses Git and is hosted at `Github <http://github.com>`_.
`Maven Repository <http://akka.io/repository/>`_
================================================
`<http://akka.io/repository>`_
The Akka Maven repository can be found at `<http://akka.io/repository>`_.
Typesafe provides `<http://repo.typesafe.com/typesafe/releases/>`_ that proxies several other repositories, including akka.io.
It is convenient to use the Typesafe repository, since it includes all external dependencies of Akka.
It is a "best-effort" service, and if it is unavailable you may need to use the underlying repositories
directly.
* http://akka.io/repository
* http://repository.codehaus.org
* http://guiceyfruit.googlecode.com/svn/repo/releases/
* http://repository.jboss.org/nexus/content/groups/public/
* http://download.java.net/maven/2
* http://oss.sonatype.org/content/repositories/releases
* http://download.java.net/maven/glassfish
* http://databinder.net/repo

View file

@ -213,8 +213,8 @@ Here is the API and how to use it from within an 'Actor':
self.link(actorRef)
self.unlink(actorRef)
// starts and links Actors atomically
self.startLink(actorRef)
// link first, then start actor
self.link(actorRef).start()
// spawns (creates and starts) actors
self.spawn[MyActor]

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