Merge master into tame-globals branch

This commit is contained in:
Peter Vlugter 2011-10-13 13:41:44 +02:00
commit 317b8bc03b
60 changed files with 2079 additions and 1696 deletions

1
.gitignore vendored
View file

@ -42,6 +42,7 @@ run-codefellow
.cache
.idea
.scala_dependencies
.cache
multiverse.log
.eprj
.*.swp

View file

@ -10,17 +10,12 @@ import static org.junit.Assert.*;
public class JavaAPI {
private AkkaApplication app = new AkkaApplication();
@Test void mustBeAbleToUseUntypedActor() {
final RemoteSupport remote = app.remote();
assertNotNull(remote);
}
@Test void mustInteractWithActorRegistry() {
final ActorRegistry registry = app.registry();
assertNotNull(registry);
}
@Test void mustBeAbleToCreateActorRefFromClass() {
ActorRef ref = app.createActor(JavaAPITestActor.class);
assertNotNull(ref);

View file

@ -285,7 +285,7 @@ class ActorRefSpec extends AkkaSpec {
val latch = new CountDownLatch(1)
val a = createActor(new InnerActor {
override def postStop {
app.registry.unregister(self)
// app.registry.unregister(self)
latch.countDown
}
})

View file

@ -12,7 +12,7 @@ import java.util.concurrent.atomic._
class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender {
"The Death Watch" must {
def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(5 seconds, "stopped") {
def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, "stopped") {
case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" true
}
@ -26,7 +26,6 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
expectTerminationOf(terminal)
terminal.stop()
expectNoMsg(2 seconds) //Shouldn't get more terminations
}
"notify with all monitors with one Terminated message when an Actor is stopped" in {
@ -46,7 +45,6 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
terminal.stop()
monitor1.stop()
monitor2.stop()
expectNoMsg(2 seconds) //Shouldn't get more terminations
}
"notify with _current_ monitors with one Terminated message when an Actor is stopped" in {
@ -67,7 +65,6 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
terminal.stop()
monitor1.stop()
monitor2.stop()
expectNoMsg(2 seconds) //Shouldn't get more terminations
}
"notify with a Terminated message once when an Actor is stopped but not when restarted" in {
@ -79,16 +76,12 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
terminal ! Kill
terminal ! Kill
terminal ! "foo"
expectMsg("foo") //Make sure that it's still alive
(terminal ? "foo").as[String] must be === Some("foo")
terminal ! Kill
expectTerminationOf(terminal)
terminal.stop()
expectNoMsg(2 seconds) //Shouldn't get more terminations
supervisor.stop()
}
}

View file

@ -5,6 +5,7 @@
package akka.actor
import akka.testkit.AkkaSpec
import akka.util.duration._
import DeploymentConfig._
class DeployerSpec extends AkkaSpec {
@ -18,9 +19,9 @@ class DeployerSpec extends AkkaSpec {
Deploy(
"service-ping",
None,
LeastCPU,
RoundRobin,
NrOfInstances(3),
BannagePeriodFailureDetector(10),
BannagePeriodFailureDetector(10 seconds),
app.deployer.deploymentConfig.RemoteScope(List(
RemoteAddress("wallace", 2552), RemoteAddress("gromit", 2552))))))
// ClusterScope(

View file

@ -44,6 +44,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
case Ping countDownLatch.countDown()
case Crash throw new Exception("Crashing...")
}
override def postRestart(reason: Throwable) = {
if (!restartLatch.isOpen)
restartLatch.open
@ -61,17 +62,17 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
slave ! Ping
// test restart and post restart ping
assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(restartLatch.tryAwait(10, TimeUnit.SECONDS))
// now crash again... should not restart
slave ! Crash
slave ! Ping
assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(countDownLatch.await(1, TimeUnit.SECONDS))
assert(secondRestartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(countDownLatch.await(10, TimeUnit.SECONDS))
slave ! Crash
assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
assert(stopLatch.tryAwait(10, TimeUnit.SECONDS))
}
"ensure that slave is immortal without max restarts and time range" in {
@ -134,14 +135,14 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
slave ! Ping
slave ! Crash
assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(pingLatch.tryAwait(1, TimeUnit.SECONDS))
assert(restartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(pingLatch.tryAwait(10, TimeUnit.SECONDS))
slave ! Ping
slave ! Crash
assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(secondPingLatch.tryAwait(1, TimeUnit.SECONDS))
assert(secondRestartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(secondPingLatch.tryAwait(10, TimeUnit.SECONDS))
// sleep to go out of the restart strategy's time range
sleep(700L)
@ -188,7 +189,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
slave ! Ping
// test restart and post restart ping
assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(restartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(!slave.isShutdown)
@ -196,13 +197,13 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
slave ! Crash
slave ! Ping
assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(countDownLatch.await(1, TimeUnit.SECONDS))
assert(secondRestartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(countDownLatch.await(10, TimeUnit.SECONDS))
sleep(700L)
slave ! Crash
assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
assert(stopLatch.tryAwait(10, TimeUnit.SECONDS))
sleep(500L)
assert(slave.isShutdown)
}
@ -238,7 +239,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
slave ! Ping
// test restart and post restart ping
assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(restartLatch.tryAwait(10, TimeUnit.SECONDS))
assert(!slave.isShutdown)
@ -247,14 +248,14 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll {
// may not be running
slave ! Ping
assert(countDownLatch.await(1, TimeUnit.SECONDS))
assert(countDownLatch.await(10, TimeUnit.SECONDS))
// may not be running
slave ! Crash
assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
assert(stopLatch.tryAwait(10, TimeUnit.SECONDS))
assert(maxNoOfRestartsLatch.tryAwait(1, TimeUnit.SECONDS))
assert(maxNoOfRestartsLatch.tryAwait(10, TimeUnit.SECONDS))
sleep(500L)
assert(slave.isShutdown)
}

View file

@ -23,7 +23,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
override def afterEach {
while (futures.peek() ne null) { Option(futures.poll()).foreach(_.cancel(true)) }
app.registry.local.shutdownAll
// app.registry.local.shutdownAll
EventHandler.start()
}
@ -67,18 +67,19 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
/**
* ticket #372
* FIXME rewrite the test so that registry is not used
*/
"not create actors" in {
object Ping
val ticks = new CountDownLatch(1000)
val actor = createActor(new Actor {
def receive = { case Ping ticks.countDown }
})
val numActors = app.registry.local.actors.length
(1 to 1000).foreach(_ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS)))
assert(ticks.await(10, TimeUnit.SECONDS))
assert(app.registry.local.actors.length === numActors)
}
// "not create actors" in {
// object Ping
// val ticks = new CountDownLatch(1000)
// val actor = createActor(new Actor {
// def receive = { case Ping ticks.countDown }
// })
// val numActors = app.registry.local.actors.length
// (1 to 1000).foreach(_ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS)))
// assert(ticks.await(10, TimeUnit.SECONDS))
// assert(app.registry.local.actors.length === numActors)
// }
/**
* ticket #372

View file

@ -16,7 +16,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender
override def beforeAll = Thread.interrupted() //remove interrupted status.
override def afterAll = {
app.registry.local.shutdownAll
// app.registry.local.shutdownAll
akka.event.EventHandler.start()
}

View file

@ -409,17 +409,18 @@ abstract class ActorModelSpec extends AkkaSpec {
} catch {
case e
app.registry.local.foreach {
case actor: LocalActorRef
val cell = actor.underlying
val mbox = cell.mailbox
System.err.println("Left in the registry: " + actor.address + " => " + cell + " => " + mbox.hasMessages + " " + mbox.hasSystemMessages + " " + mbox.numberOfMessages + " " + mbox.isScheduled)
var message = mbox.dequeue()
while (message ne null) {
System.err.println("Lingering message for " + cell + " " + message)
message = mbox.dequeue()
}
}
// FIXME: registry has been removed
// app.registry.local.foreach {
// case actor: LocalActorRef
// val cell = actor.underlying
// val mbox = cell.mailbox
// System.err.println("Left in the registry: " + actor.address + " => " + cell + " => " + mbox.hasMessages + " " + mbox.hasSystemMessages + " " + mbox.numberOfMessages + " " + mbox.isScheduled)
// var message = mbox.dequeue()
// while (message ne null) {
// System.err.println("Lingering message for " + cell + " " + message)
// message = mbox.dequeue()
// }
// }
throw e
}

View file

@ -8,7 +8,7 @@ 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 }
import akka.actor.{ LocalActorRef, Actor, NullChannel }
import akka.testkit.AkkaSpec
@RunWith(classOf[JUnitRunner])

View file

@ -0,0 +1,223 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.event
import org.scalatest.BeforeAndAfterEach
import akka.testkit._
import akka.util.duration._
import java.util.concurrent.atomic._
import akka.actor.{ Props, Actor, ActorRef }
import java.util.Comparator
import akka.japi.{ Procedure, Function }
object EventBusSpec {
class TestActorWrapperActor(testActor: ActorRef) extends Actor {
def receive = {
case x testActor forward x
}
}
}
abstract class EventBusSpec(busName: String) extends AkkaSpec with BeforeAndAfterEach {
import EventBusSpec._
type BusType <: EventBus
def createNewEventBus(): BusType
def createEvents(numberOfEvents: Int): Iterable[BusType#Event]
def createSubscriber(pipeTo: ActorRef): BusType#Subscriber
def classifierFor(event: BusType#Event): BusType#Classifier
def disposeSubscriber(subscriber: BusType#Subscriber): Unit
busName must {
def createNewSubscriber() = createSubscriber(testActor).asInstanceOf[bus.Subscriber]
def getClassifierFor(event: BusType#Event) = classifierFor(event).asInstanceOf[bus.Classifier]
def createNewEvents(numberOfEvents: Int): Iterable[bus.Event] = createEvents(numberOfEvents).asInstanceOf[Iterable[bus.Event]]
val bus = createNewEventBus()
val events = createNewEvents(100)
val event = events.head
val classifier = getClassifierFor(event)
val subscriber = createNewSubscriber()
"allow subscribers" in {
bus.subscribe(subscriber, classifier) must be === true
}
"allow to unsubscribe already existing subscriber" in {
bus.unsubscribe(subscriber, classifier) must be === true
}
"not allow to unsubscribe non-existing subscriber" in {
val sub = createNewSubscriber()
bus.unsubscribe(sub, classifier) must be === false
disposeSubscriber(sub)
}
"not allow for the same subscriber to subscribe to the same channel twice" in {
bus.subscribe(subscriber, classifier) must be === true
bus.subscribe(subscriber, classifier) must be === false
bus.unsubscribe(subscriber, classifier) must be === true
}
"not allow for the same subscriber to unsubscribe to the same channel twice" in {
bus.subscribe(subscriber, classifier) must be === true
bus.unsubscribe(subscriber, classifier) must be === true
bus.unsubscribe(subscriber, classifier) must be === false
}
"allow to add multiple subscribers" in {
val subscribers = (1 to 10) map { _ createNewSubscriber() }
val events = createEvents(10)
val classifiers = events map getClassifierFor
subscribers.zip(classifiers) forall { case (s, c) bus.subscribe(s, c) } must be === true
subscribers.zip(classifiers) forall { case (s, c) bus.unsubscribe(s, c) } must be === true
subscribers foreach disposeSubscriber
}
"publishing events without any subscribers shouldn't be a problem" in {
bus.publish(event)
}
"publish the given event to the only subscriber" in {
bus.subscribe(subscriber, classifier)
bus.publish(event)
expectMsg(event)
expectNoMsg(1 second)
bus.unsubscribe(subscriber, classifier)
}
"publish to the only subscriber multiple times" in {
bus.subscribe(subscriber, classifier)
bus.publish(event)
bus.publish(event)
bus.publish(event)
expectMsg(event)
expectMsg(event)
expectMsg(event)
expectNoMsg(1 second)
bus.unsubscribe(subscriber, classifier)
}
"publish the given event to all intended subscribers" in {
val range = 0 until 10
val subscribers = range map (_ createNewSubscriber())
subscribers foreach { s bus.subscribe(s, classifier) must be === true }
bus.publish(event)
range foreach { _ expectMsg(event) }
subscribers foreach { s bus.unsubscribe(s, classifier) must be === true; disposeSubscriber(s) }
}
"not publish the given event to any other subscribers than the intended ones" in {
val otherSubscriber = createNewSubscriber()
val otherClassifier = getClassifierFor(events.drop(1).head)
bus.subscribe(subscriber, classifier)
bus.subscribe(otherSubscriber, otherClassifier)
bus.publish(event)
expectMsg(event)
bus.unsubscribe(subscriber, classifier)
bus.unsubscribe(otherSubscriber, otherClassifier)
expectNoMsg(1 second)
}
"not publish the given event to a former subscriber" in {
bus.subscribe(subscriber, classifier)
bus.unsubscribe(subscriber, classifier)
bus.publish(event)
expectNoMsg(1 second)
}
"cleanup subscriber" in {
disposeSubscriber(subscriber)
}
}
}
object ActorEventBusSpec {
class ComposedActorEventBus extends ActorEventBus with LookupClassification {
type Event = Int
type Classifier = String
def classify(event: Event) = event.toString
protected def compareSubscribers(a: Subscriber, b: Subscriber): Int = a compareTo b
protected def mapSize = 32
def publish(event: Event, subscriber: Subscriber) = subscriber ! event
}
}
class ActorEventBusSpec extends EventBusSpec("ActorEventBus") {
import akka.event.ActorEventBusSpec._
import EventBusSpec.TestActorWrapperActor
type BusType = ComposedActorEventBus
def createNewEventBus(): BusType = new ComposedActorEventBus
def createEvents(numberOfEvents: Int) = (0 until numberOfEvents)
def createSubscriber(pipeTo: ActorRef) = createActor(Props(new TestActorWrapperActor(pipeTo)))
def classifierFor(event: BusType#Event) = event.toString
def disposeSubscriber(subscriber: BusType#Subscriber): Unit = subscriber.stop()
}
object ScanningEventBusSpec {
import akka.event.japi.ScanningEventBus
class MyScanningEventBus extends ScanningEventBus[Int, akka.japi.Procedure[Int], String] {
protected def compareClassifiers(a: Classifier, b: Classifier): Int = a compareTo b
protected def compareSubscribers(a: Subscriber, b: Subscriber): Int = System.identityHashCode(a) - System.identityHashCode(b)
protected def matches(classifier: Classifier, event: Event): Boolean = event.toString == classifier
protected def publish(event: Event, subscriber: Subscriber): Unit = subscriber(event)
}
}
class ScanningEventBusSpec extends EventBusSpec("ScanningEventBus") {
import ScanningEventBusSpec._
type BusType = MyScanningEventBus
def createNewEventBus(): BusType = new MyScanningEventBus
def createEvents(numberOfEvents: Int) = (0 until numberOfEvents)
def createSubscriber(pipeTo: ActorRef) = new Procedure[Int] { def apply(i: Int) = pipeTo ! i }
def classifierFor(event: BusType#Event) = event.toString
def disposeSubscriber(subscriber: BusType#Subscriber): Unit = ()
}
object LookupEventBusSpec {
class MyLookupEventBus extends akka.event.japi.LookupEventBus[Int, akka.japi.Procedure[Int], String] {
protected def classify(event: Event): Classifier = event.toString
protected def compareSubscribers(a: Subscriber, b: Subscriber): Int = System.identityHashCode(a) - System.identityHashCode(b)
protected def mapSize = 32
protected def publish(event: Event, subscriber: Subscriber): Unit = subscriber(event)
}
}
class LookupEventBusSpec extends EventBusSpec("LookupEventBus") {
import LookupEventBusSpec._
type BusType = MyLookupEventBus
def createNewEventBus(): BusType = new MyLookupEventBus
def createEvents(numberOfEvents: Int) = (0 until numberOfEvents)
def createSubscriber(pipeTo: ActorRef) = new Procedure[Int] { def apply(i: Int) = pipeTo ! i }
def classifierFor(event: BusType#Event) = event.toString
def disposeSubscriber(subscriber: BusType#Subscriber): Unit = ()
}

View file

@ -34,7 +34,7 @@ abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkSc
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs)).withDispatcher(clientDispatcher)
Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), sampling, delayMs)).withDispatcher(clientDispatcher)
}).toList.map(app.createActor(_))
clients.foreach(_ ! "run")
@ -47,28 +47,35 @@ abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkSc
clients.foreach(_ ! PoisonPill)
}
class Client(orderReceiver: ActorRef, orders: List[Order], latch: CountDownLatch, repeat: Int, delayMs: Int) extends Actor {
def this(orderReceiver: ActorRef, orders: List[Order], latch: CountDownLatch, repeat: Int) {
this(orderReceiver, orders, latch, repeat, 0)
}
class Client(
orderReceiver: ActorRef,
orders: List[Order],
latch: CountDownLatch,
repeat: Int,
sampling: Int,
delayMs: Int = 0) extends Actor {
def receive = {
case "run"
(1 to repeat).foreach(i
{
for (o orders) {
var n = 0
for (r 1 to repeat; o orders) {
n += 1
val rsp =
if (n % sampling == 0) {
val t0 = System.nanoTime
val rsp = placeOrder(orderReceiver, o)
val duration = System.nanoTime - t0
stat.addValue(duration)
if (!rsp.status) {
EventHandler.error(this, "Invalid rsp")
}
delay(delayMs)
rsp
} else {
placeOrder(orderReceiver, o)
}
})
if (!rsp.status) {
EventHandler.error(this, "Invalid rsp")
}
delay(delayMs)
}
latch.countDown()
}
}

View file

@ -49,6 +49,10 @@ trait PerformanceTest extends JUnitSuite {
System.getProperty("benchmark.timeDilation", "1").toLong
}
def sampling = {
System.getProperty("benchmark.sampling", "100").toInt
}
var stat: DescriptiveStatistics = _
val resultRepository = BenchResultRepository()
@ -113,16 +117,18 @@ trait PerformanceTest extends JUnitSuite {
75 -> (stat.getPercentile(75.0) / 1000).toLong,
95 -> (stat.getPercentile(95.0) / 1000).toLong)
val n = stat.getN * sampling
val stats = Stats(
name,
load = numberOfClients,
timestamp = TestStart.startTime,
durationNanos = durationNs,
n = stat.getN,
n = n,
min = (stat.getMin / 1000).toLong,
max = (stat.getMax / 1000).toLong,
mean = (stat.getMean / 1000).toLong,
tps = (stat.getN.toDouble / durationS),
tps = (n.toDouble / durationS),
percentiles)
resultRepository.add(stats)

View file

@ -21,7 +21,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
None,
RoundRobin,
NrOfInstances(5),
RemoveConnectionOnFirstFailureLocalFailureDetector,
NoOpFailureDetector,
LocalScope))
val helloLatch = new CountDownLatch(5)
@ -57,7 +57,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
None,
RoundRobin,
NrOfInstances(10),
RemoveConnectionOnFirstFailureLocalFailureDetector,
NoOpFailureDetector,
LocalScope))
val connectionCount = 10
@ -102,7 +102,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
None,
RoundRobin,
NrOfInstances(5),
RemoveConnectionOnFirstFailureLocalFailureDetector,
NoOpFailureDetector,
LocalScope))
val helloLatch = new CountDownLatch(5)
@ -137,7 +137,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
None,
Random,
NrOfInstances(7),
RemoveConnectionOnFirstFailureLocalFailureDetector,
NoOpFailureDetector,
LocalScope))
val stopLatch = new CountDownLatch(7)
@ -171,7 +171,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
None,
Random,
NrOfInstances(10),
RemoveConnectionOnFirstFailureLocalFailureDetector,
NoOpFailureDetector,
LocalScope))
val connectionCount = 10
@ -216,7 +216,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
None,
Random,
NrOfInstances(6),
RemoveConnectionOnFirstFailureLocalFailureDetector,
NoOpFailureDetector,
LocalScope))
val helloLatch = new CountDownLatch(6)

View file

@ -1,12 +1,13 @@
package akka.routing
import akka.routing._
import akka.config.ConfigurationException
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.{ ActorRef, Actor }
import collection.mutable.LinkedList
import akka.routing.Routing.Broadcast
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import akka.testkit.AkkaSpec
import akka.testkit._
object RoutingSpec {
@ -26,18 +27,18 @@ class RoutingSpec extends AkkaSpec {
"be started when constructed" in {
val actor1 = createActor[TestActor]
val props = RoutedProps(() new DirectRouter, List(actor1))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withDirectRouter.withLocalConnections(List(actor1))
val actor = app.createActor(props, "foo")
actor.isShutdown must be(false)
}
"throw IllegalArgumentException at construction when no connections" in {
"throw ConfigurationException at construction when no connections" in {
try {
val props = RoutedProps(() new DirectRouter, List())
app.routing.actorOf(props, "foo")
val props = RoutedProps().withDirectRouter
app.createActor(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
case e: ConfigurationException
}
}
@ -52,8 +53,8 @@ class RoutingSpec extends AkkaSpec {
}
})
val props = RoutedProps(() new DirectRouter, List(connection1))
val routedActor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withDirectRouter.withLocalConnections(List(connection1))
val routedActor = app.createActor(props, "foo")
routedActor ! "hello"
routedActor ! "end"
@ -73,8 +74,8 @@ class RoutingSpec extends AkkaSpec {
}
})
val props = RoutedProps(() new DirectRouter, List(connection1))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withDirectRouter.withLocalConnections(List(connection1))
val actor = app.createActor(props, "foo")
actor ! Broadcast(1)
actor ! "end"
@ -90,18 +91,18 @@ class RoutingSpec extends AkkaSpec {
"be started when constructed" in {
val actor1 = createActor[TestActor]
val props = RoutedProps(() new RoundRobinRouter, List(actor1))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(actor1))
val actor = app.createActor(props, "foo")
actor.isShutdown must be(false)
}
"throw IllegalArgumentException at construction when no connections" in {
"throw ConfigurationException at construction when no connections" in {
try {
val props = RoutedProps(() new RoundRobinRouter, List())
app.routing.actorOf(props, "foo")
val props = RoutedProps().withRoundRobinRouter
app.createActor(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
case e: ConfigurationException
}
}
@ -130,8 +131,8 @@ class RoutingSpec extends AkkaSpec {
}
//create the routed actor.
val props = RoutedProps(() new RoundRobinRouter, connections)
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRoundRobinRouter.withLocalConnections(connections)
val actor = app.createActor(props, "foo")
//send messages to the actor.
for (i 0 until iterationCount) {
@ -169,8 +170,8 @@ class RoutingSpec extends AkkaSpec {
}
})
val props = RoutedProps(() new RoundRobinRouter, List(connection1, connection2))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(connection1, connection2))
val actor = app.createActor(props, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -192,8 +193,8 @@ class RoutingSpec extends AkkaSpec {
}
})
val props = RoutedProps(() new RoundRobinRouter, List(connection1))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(connection1))
val actor = app.createActor(props, "foo")
try {
actor ? Broadcast(1)
@ -214,18 +215,18 @@ class RoutingSpec extends AkkaSpec {
val actor1 = createActor[TestActor]
val props = RoutedProps(() new RandomRouter, List(actor1))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRandomRouter.withLocalConnections(List(actor1))
val actor = app.createActor(props, "foo")
actor.isShutdown must be(false)
}
"throw IllegalArgumentException at construction when no connections" in {
"throw ConfigurationException at construction when no connections" in {
try {
val props = RoutedProps(() new RandomRouter, List())
app.routing.actorOf(props, "foo")
val props = RoutedProps().withRandomRouter
app.createActor(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
case e: ConfigurationException
}
}
@ -252,8 +253,8 @@ class RoutingSpec extends AkkaSpec {
}
})
val props = RoutedProps(() new RandomRouter, List(connection1, connection2))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRandomRouter.withLocalConnections(List(connection1, connection2))
val actor = app.createActor(props, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -275,8 +276,8 @@ class RoutingSpec extends AkkaSpec {
}
})
val props = RoutedProps(() new RandomRouter, List(connection1))
val actor = app.routing.actorOf(props, "foo")
val props = RoutedProps().withRandomRouter.withLocalConnections(List(connection1))
val actor = app.createActor(props, "foo")
try {
actor ? Broadcast(1)
@ -290,4 +291,179 @@ class RoutingSpec extends AkkaSpec {
counter1.get must be(0)
}
}
"Scatter-gather router" must {
"return response, even if one of the connections has stopped" in {
val shutdownLatch = new TestLatch(1)
val props = RoutedProps()
.withLocalConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
actor ! Broadcast(Stop(Some(0)))
shutdownLatch.await
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"throw an exception, if all the connections have stopped" in {
val shutdownLatch = new TestLatch(2)
val props = RoutedProps()
.withLocalConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
actor ! Broadcast(Stop())
shutdownLatch.await
(intercept[RoutingException] {
actor ? Broadcast(0)
}) must not be (null)
}
"return the first response from connections, when all of them replied" in {
val props = RoutedProps()
.withLocalConnections(List(newActor(0), newActor(1)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
(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()
.withLocalConnections(List(newActor(0), newActor(1)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"be started when constructed" in {
val props = RoutedProps()
.withLocalConnections(List(newActor(0)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
actor.isShutdown must be(false)
}
"throw ConfigurationException at construction when no connections" in {
val props = RoutedProps()
.withLocalConnections(List())
.withRouter(() new ScatterGatherFirstCompletedRouter())
try {
app.createActor(props, "foo")
fail()
} catch {
case e: ConfigurationException
}
}
"deliver one-way messages in a round robin fashion" in {
val connectionCount = 10
val iterationCount = 10
val doneLatch = new TestLatch(connectionCount)
var connections = new LinkedList[ActorRef]
var counters = new LinkedList[AtomicInteger]
for (i 0 until connectionCount) {
counters = counters :+ new AtomicInteger()
val connection = app.createActor(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg)
}
})
connections = connections :+ connection
}
val props = RoutedProps()
.withLocalConnections(connections)
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
for (i 0 until iterationCount) {
for (k 0 until connectionCount) {
actor ! (k + 1)
}
}
actor ! Broadcast("end")
doneLatch.await
for (i 0 until connectionCount) {
val counter = counters.get(i).get
counter.get must be((iterationCount * (i + 1)))
}
}
"deliver a broadcast message using the !" in {
val doneLatch = new TestLatch(2)
val counter1 = new AtomicInteger
val connection1 = app.createActor(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
})
val counter2 = new AtomicInteger
val connection2 = app.createActor(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
})
val props = RoutedProps.apply()
.withLocalConnections(List(connection1, connection2))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.createActor(props, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
doneLatch.await
counter1.get must be(1)
counter2.get must be(1)
}
case class Stop(id: Option[Int] = None)
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = app.createActor(new Actor {
def receive = {
case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) self.stop()
case _id: Int if (_id == id)
case _ Thread sleep 100 * id; tryReply(id)
}
override def postStop = {
shudownLatch foreach (_.countDown())
}
})
}
}

View file

@ -1,189 +0,0 @@
package akka.ticket
import akka.routing._
import akka.actor.{ ActorRef, Actor }
import java.util.concurrent.atomic.AtomicInteger
import collection.mutable.LinkedList
import akka.routing.Routing.Broadcast
import akka.testkit._
class Ticket1111Spec extends AkkaSpec {
"Scatter-gather router" must {
"return response, even if one of the connections has stopped" in {
val shutdownLatch = new TestLatch(1)
val props = RoutedProps()
.withConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
actor ! Broadcast(Stop(Some(0)))
shutdownLatch.await
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"throw an exception, if all the connections have stopped" in {
val shutdownLatch = new TestLatch(2)
val props = RoutedProps()
.withConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
actor ! Broadcast(Stop())
shutdownLatch.await
(intercept[RoutingException] {
actor ? Broadcast(0)
}) must not be (null)
}
"return the first response from connections, when all of them replied" in {
val props = RoutedProps()
.withConnections(List(newActor(0), newActor(1)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
(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()
.withConnections(List(newActor(0), newActor(1)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
}
"be started when constructed" in {
val props = RoutedProps()
.withConnections(List(newActor(0)))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
actor.isShutdown must be(false)
}
"throw IllegalArgumentException at construction when no connections" in {
val props = RoutedProps()
.withConnections(List())
.withRouter(() new ScatterGatherFirstCompletedRouter())
try {
app.routing.actorOf(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
}
}
"deliver one-way messages in a round robin fashion" in {
val connectionCount = 10
val iterationCount = 10
val doneLatch = new TestLatch(connectionCount)
var connections = new LinkedList[ActorRef]
var counters = new LinkedList[AtomicInteger]
for (i 0 until connectionCount) {
counters = counters :+ new AtomicInteger()
val connection = createActor(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg)
}
})
connections = connections :+ connection
}
val props = RoutedProps()
.withConnections(connections)
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
for (i 0 until iterationCount) {
for (k 0 until connectionCount) {
actor ! (k + 1)
}
}
actor ! Broadcast("end")
doneLatch.await
for (i 0 until connectionCount) {
val counter = counters.get(i).get
counter.get must be((iterationCount * (i + 1)))
}
}
"deliver a broadcast message using the !" in {
val doneLatch = new TestLatch(2)
val counter1 = new AtomicInteger
val connection1 = createActor(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
})
val counter2 = new AtomicInteger
val connection2 = createActor(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
})
val props = RoutedProps.apply()
.withConnections(List(connection1, connection2))
.withRouter(() new ScatterGatherFirstCompletedRouter())
val actor = app.routing.actorOf(props, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
doneLatch.await
counter1.get must be(1)
counter2.get must be(1)
}
case class Stop(id: Option[Int] = None)
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = createActor(new Actor {
def receive = {
case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) self.stop()
case _id: Int if (_id == id)
case _ Thread sleep 100 * id; tryReply(id)
}
override def postStop = {
shudownLatch foreach (_.countDown())
}
})
}
}

View file

@ -16,7 +16,6 @@ import com.eaio.uuid.UUID;
* - locating actors
*/
public class Actors {
/**
* The message that is sent when an Actor gets a receive timeout.
* <pre>

View file

@ -156,12 +156,12 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
* Handle to the ActorRegistry.
* TODO: delete me!
*/
val registry = new ActorRegistry
// val registry = new ActorRegistry
// TODO check memory consistency issues
val reflective = new ReflectiveAccess(this)
val routing = new Routing(this)
// val routing = new Routing(this)
val remote = reflective.RemoteModule.defaultRemoteSupport map (_.apply) getOrElse null

View file

@ -164,7 +164,6 @@ object Actor {
case _ new LoggingReceive(source, r)
}
}
}
/**

View file

@ -251,7 +251,6 @@ private[akka] class ActorCell(
}
}
application.registry.register(self)
dispatcher.attach(this)
}
@ -385,7 +384,6 @@ private[akka] class ActorCell(
receiveTimeout = None
cancelReceiveTimeout
application.provider.evict(self.address)
application.registry.unregister(self)
dispatcher.detach(this)
try {
@ -405,7 +403,7 @@ private[akka] class ActorCell(
if (supervisor.isDefined) supervisor.get ! ChildTerminated(self, cause)
InVMMonitoring.signal(Terminated(self, cause))
InVMMonitoring.publish(Terminated(self, cause))
currentMessage = null
clearActorContext()
@ -427,10 +425,10 @@ private[akka] class ActorCell(
case Create create()
case Recreate(cause) recreate(cause)
case Link(subject)
akka.event.InVMMonitoring.link(self, subject)
akka.event.InVMMonitoring.subscribe(self, subject)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "now monitoring " + subject)
case Unlink(subject)
akka.event.InVMMonitoring.unlink(self, subject)
akka.event.InVMMonitoring.unsubscribe(self, subject)
if (application.AkkaConfig.DebugLifecycle) EventHandler.debug(actor, "stopped monitoring " + subject)
case Suspend suspend()
case Resume resume()

View file

@ -13,6 +13,7 @@ import java.lang.{ UnsupportedOperationException, IllegalStateException }
import akka.AkkaApplication
import akka.remote.RemoteSupport
import scala.util.DynamicVariable
import akka.event.{ EventHandler, InVMMonitoring }
/**
* ActorRef is an immutable and serializable handle to an Actor.
@ -263,7 +264,6 @@ object RemoteActorSystemMessage {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[akka] case class RemoteActorRef private[akka] (
val application: AkkaApplication,
val remote: RemoteSupport,
val remoteAddress: InetSocketAddress,
val address: String,
@ -275,8 +275,6 @@ private[akka] case class RemoteActorRef private[akka] (
def isShutdown: Boolean = !running
application.reflective.RemoteModule.ensureEnabled()
def postMessageToMailbox(message: Any, channel: UntypedChannel) {
val chSender = if (channel.isInstanceOf[ActorRef]) Some(channel.asInstanceOf[ActorRef]) else None
remote.send[Any](message, chSender, None, remoteAddress, true, this, loader)
@ -392,13 +390,14 @@ case class SerializedActorRef(uuid: Uuid,
if (application.value eq null) throw new IllegalStateException(
"Trying to deserialize a serialized ActorRef without an AkkaApplication in scope." +
" Use akka.serialization.Serialization.application.withValue(akkaApplication) { ... }")
application.value.registry.local.actorFor(uuid) match {
application.value.provider.actorFor(address) match {
case Some(actor) actor
case None
//TODO FIXME Add case for when hostname+port == remote.address.hostname+port, should return a DeadActorRef or something
// TODO FIXME Add case for when hostname+port == remote.address.hostname+port, should return a DeadActorRef or something
// TODO FIXME the remote should only be in the remote actor ref provider
val remote = application.value.reflective.RemoteModule
if (remote.isEnabled)
RemoteActorRef(application.value, remote.defaultRemoteSupport.get(), new InetSocketAddress(hostname, port), address, None)
RemoteActorRef(remote.defaultRemoteSupport.get(), new InetSocketAddress(hostname, port), address, None)
else
throw new IllegalStateException(
"Trying to deserialize ActorRef [" + this +
@ -424,3 +423,23 @@ trait UnsupportedActorRef extends ActorRef with ScalaActorRef {
private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
}
class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
val address: String = "akka:internal:DeadLetterActorRef"
override def link(actorRef: ActorRef): ActorRef = actorRef
override def unlink(actorRef: ActorRef): ActorRef = actorRef
def isShutdown(): Boolean = true
def stop(): Unit = ()
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = EventHandler.debug(this, message)
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = { EventHandler.debug(this, message); brokenPromise }
}

View file

@ -4,15 +4,16 @@
package akka.actor
import DeploymentConfig._
import akka.event.EventHandler
import akka.AkkaException
import akka.config.ConfigurationException
import akka.util.ReflectiveAccess
import akka.routing._
import akka.AkkaApplication
import akka.dispatch.MessageDispatcher
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
import com.eaio.uuid.UUID
import akka.AkkaException
/**
* Interface for all ActorRef providers to implement.
@ -21,7 +22,9 @@ trait ActorRefProvider {
def actorOf(props: Props, address: String): Option[ActorRef]
def findActorRef(address: String): Option[ActorRef]
def actorOf(props: RoutedProps, address: String): Option[ActorRef]
def actorFor(address: String): Option[ActorRef]
private[akka] def evict(address: String): Boolean
}
@ -52,7 +55,11 @@ trait ActorRefFactory {
def createActor(creator: UntypedActorFactory): ActorRef = createActor(Props(() creator.create()))
def findActor(address: String): Option[ActorRef] = provider.findActorRef(address)
def createActor(props: RoutedProps): ActorRef = createActor(props, new UUID().toString)
def createActor(props: RoutedProps, address: String): ActorRef = provider.actorOf(props, address).get
def findActor(address: String): Option[ActorRef] = provider.actorFor(address)
}
@ -74,7 +81,10 @@ class LocalActorRefProvider(val application: AkkaApplication, val deployer: Depl
def actorOf(props: Props, address: String): Option[ActorRef] = actorOf(props, address, false)
def findActorRef(address: String): Option[ActorRef] = application.registry.local.actorFor(address)
def actorFor(address: String): Option[ActorRef] = actors.get(address) match {
case null None
case future future.await.resultOrException.getOrElse(None)
}
/**
* Returns true if the actor was in the provider's cache and evicted successfully, else false.
@ -90,6 +100,8 @@ class LocalActorRefProvider(val application: AkkaApplication, val deployer: Depl
else
props
val defaultTimeout = application.AkkaConfig.ActorTimeout
val newFuture = Promise[Option[ActorRef]](5000)(application.dispatcher) // FIXME is this proper timeout?
val oldFuture = actors.putIfAbsent(address, newFuture)
@ -99,30 +111,28 @@ class LocalActorRefProvider(val application: AkkaApplication, val deployer: Depl
deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor
// create a local actor
case None | Some(Deploy(_, _, Direct, _, _, LocalScope))
case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope))
Some(new LocalActorRef(application, localProps, address, systemService)) // create a local actor
// create a routed actor ref
case deploy @ Some(Deploy(_, _, router, nrOfInstances, _, LocalScope))
val routerType = DeploymentConfig.routerTypeFor(router)
val routerFactory: () Router = routerType match {
case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
case RouterType.Custom sys.error("Router Custom not supported yet")
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope))
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()(props.dispatcher, defaultTimeout)
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
case RouterType.Custom(implClass) () Routing.createCustomRouter(implClass)
}
val connections: Iterable[ActorRef] =
if (nrOfInstances.factor > 0)
Vector.fill(nrOfInstances.factor)(new LocalActorRef(application, localProps, new UUID().toString, systemService))
else Nil
Some(application.routing.actorOf(RoutedProps(
routerFactory = routerFactory,
connections = connections)))
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), address)
case _ None // non-local actor - pass it on
}
@ -132,8 +142,6 @@ class LocalActorRefProvider(val application: AkkaApplication, val deployer: Depl
throw e
}
actor foreach application.registry.register // only for ActorRegistry backward compat, will be removed later
newFuture completeWithResult actor
actor
@ -141,4 +149,21 @@ class LocalActorRefProvider(val application: AkkaApplication, val deployer: Depl
oldFuture.await.resultOrException.getOrElse(None)
}
}
/**
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
*/
def actorOf(props: RoutedProps, address: String): Option[ActorRef] = {
//FIXME clustering should be implemented by cluster actor ref provider
//TODO Implement support for configuring by deployment ID etc
//TODO If address matches an already created actor (Ahead-of-time deployed) return that actor
//TODO If address 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?
if (props.connectionManager.size == 0) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router")
// val clusteringEnabled = ReflectiveAccess.ClusterModule.isEnabled
// val localOnly = props.localOnly
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
// else new RoutedActorRef(props, address)
Some(new RoutedActorRef(props, address))
}
}

View file

@ -1,299 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import scala.collection.mutable.ListBuffer
import java.util.concurrent.ConcurrentHashMap
import akka.util.ListenerManagement
import reflect.BeanProperty
/**
* Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
sealed trait ActorRegistryEvent
case class ActorRegistered(@BeanProperty address: String, @BeanProperty actor: ActorRef) extends ActorRegistryEvent
case class ActorUnregistered(@BeanProperty address: String, @BeanProperty actor: ActorRef) extends ActorRegistryEvent
case class TypedActorRegistered(@BeanProperty address: String, @BeanProperty actor: ActorRef, @BeanProperty proxy: AnyRef) extends ActorRegistryEvent
case class TypedActorUnregistered(@BeanProperty address: String, @BeanProperty actor: ActorRef, @BeanProperty proxy: AnyRef) extends ActorRegistryEvent
/**
* Registry holding all Actor instances in the whole system.
* Mapped by address which is a unique string.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[akka] final class ActorRegistry private[akka] () extends ListenerManagement {
private val actorsByAddress = new ConcurrentHashMap[String, ActorRef]
private val actorsByUuid = new ConcurrentHashMap[Uuid, ActorRef]
private val typedActorsByUuid = new ConcurrentHashMap[Uuid, AnyRef]
val local = new LocalActorRegistry(actorsByAddress, actorsByUuid, typedActorsByUuid)
/**
* Finds the actor that has a specific address.
*/
def actorFor(address: String): Option[ActorRef] = Option(actorsByAddress.get(address))
/**
* Finds the typed actors that have a specific address.
*/
def typedActorFor(address: String): Option[AnyRef] =
actorFor(address) map (typedActorFor(_))
/**
* Registers an actor in the ActorRegistry.
*/
private[akka] def register(actor: ActorRef) {
val address = actor.address
// FIXME: this check is nice but makes serialization/deserialization specs break
//if (actorsByAddress.containsKey(address) || registeredInCluster(address))
// throw new IllegalStateException("Actor 'address' [" + address + "] is already in use, can't register actor [" + actor + "]")
actorsByAddress.put(address, actor)
actorsByUuid.put(actor.uuid, actor)
notifyListeners(ActorRegistered(address, actor))
}
private[akka] def registerTypedActor(actorRef: ActorRef, proxy: AnyRef) {
if (typedActorsByUuid.putIfAbsent(actorRef.uuid, proxy) eq null)
notifyListeners(TypedActorRegistered(actorRef.address, actorRef, proxy))
}
private[akka] def unregisterTypedActor(actorRef: ActorRef, proxy: AnyRef) {
if (typedActorsByUuid.remove(actorRef.uuid, proxy))
notifyListeners(TypedActorUnregistered(actorRef.address, actorRef, proxy))
}
/**
* Unregisters an actor in the ActorRegistry.
*/
private[akka] def unregister(address: String) {
val actor = actorsByAddress remove address
actorsByUuid remove actor.uuid
notifyListeners(ActorUnregistered(address, actor))
}
/**
* Unregisters an actor in the ActorRegistry.
*/
private[akka] def unregister(actor: ActorRef) {
val address = actor.address
actorsByAddress remove address
actorsByUuid remove actor.uuid
notifyListeners(ActorUnregistered(address, actor))
//Safe cleanup (if called from the outside)
val proxy = typedActorsByUuid.remove(actor.uuid)
if (proxy ne null)
notifyListeners(TypedActorUnregistered(address, actor, proxy))
}
/**
* Registers an actor in the Cluster ActorRegistry.
*/
// private[akka] def registerInCluster[T <: Actor](
// address: String, actorRef: ActorRef, replicas: Int, serializeMailbox: Boolean = false)(implicit format: Serializer) {
// // FIXME: implement ActorRegistry.registerInCluster(..)
// }
/**
* Unregisters an actor in the Cluster ActorRegistry.
*/
// private[akka] def unregisterInCluster(address: String) {
// ClusterModule.node.remove(address)
// }
/**
* Get the typed actor proxy for a given typed actor ref.
*/
private def typedActorFor(actorRef: ActorRef): Option[AnyRef] =
Option(typedActorsByUuid.get(actorRef.uuid))
}
/**
* Projection over the local actor registry.
*/
class LocalActorRegistry(
private val actorsByAddress: ConcurrentHashMap[String, ActorRef],
private val actorsByUuid: ConcurrentHashMap[Uuid, ActorRef],
private val typedActorsByUuid: ConcurrentHashMap[Uuid, AnyRef]) {
// NOTE: currently ClusterActorRef's are only taken into account in 'actorFor(..)' - not in 'find', 'filter' etc.
private val clusterActorRefsByAddress = new ConcurrentHashMap[String, ActorRef]
private val clusterActorRefsByUuid = new ConcurrentHashMap[Uuid, ActorRef]
/**
* Returns the number of actors in the system.
*/
def size: Int = actorsByAddress.size
/**
* Shuts down and unregisters all actors in the system.
*/
def shutdownAll() {
foreach(_.stop)
actorsByAddress.clear()
actorsByUuid.clear()
typedActorsByUuid.clear()
}
//============== ACTORS ==============
/**
* Finds the actor that have a specific address.
*
* If a ClusterActorRef exists in the registry, then return that before we look after a LocalActorRef.
*/
def actorFor(address: String): Option[ActorRef] = {
if (clusterActorRefsByAddress.containsKey(address)) Some(clusterActorRefsByAddress.get(address))
else if (actorsByAddress.containsKey(address)) Some(actorsByAddress.get(address))
else None
}
private[akka] def actorFor(uuid: Uuid): Option[ActorRef] =
if (clusterActorRefsByUuid.containsKey(uuid)) Some(clusterActorRefsByUuid.get(uuid))
else if (actorsByUuid.containsKey(uuid)) Some(actorsByUuid.get(uuid))
else None
// By-passes checking the registry for ClusterActorRef and only returns possible LocalActorRefs
private[akka] def localActorRefFor(address: String): Option[ActorRef] = {
if (actorsByAddress.containsKey(address)) Some(actorsByAddress.get(address))
else None
}
// By-passes checking the registry for ClusterActorRef and only returns possible LocalActorRefs
private[akka] def localActorRefFor(uuid: Uuid): Option[ActorRef] =
if (actorsByUuid.containsKey(uuid)) Some(actorsByUuid.get(uuid))
else None
/**
* Finds the typed actor that have a specific address.
*/
def typedActorFor(address: String): Option[AnyRef] =
actorFor(address) map (typedActorFor(_)) getOrElse None
/**
* Finds the typed actor that have a specific uuid.
*/
private[akka] def typedActorFor(uuid: Uuid): Option[AnyRef] =
Option(typedActorsByUuid.get(uuid))
/**
* Returns all actors in the system.
*/
def actors: Array[ActorRef] = filter(_ true)
/**
* Invokes a function for all actors.
*/
def foreach(f: (ActorRef) Unit) = {
val elements = actorsByAddress.elements
while (elements.hasMoreElements) f(elements.nextElement)
}
/**
* Invokes the function on all known actors until it returns Some
* Returns None if the function never returns Some
*/
def find[T](f: PartialFunction[ActorRef, T]): Option[T] = {
val elements = actorsByAddress.elements
while (elements.hasMoreElements) {
val element = elements.nextElement
if (f isDefinedAt element) return Some(f(element))
}
None
}
/**
* Finds all actors that satisfy a predicate.
*/
def filter(p: ActorRef Boolean): Array[ActorRef] = {
val all = new ListBuffer[ActorRef]
val elements = actorsByAddress.elements
while (elements.hasMoreElements) {
val actorId = elements.nextElement
if (p(actorId)) all += actorId
}
all.toArray
}
//============== TYPED ACTORS ==============
/**
* Returns all typed actors in the system.
*/
def typedActors: Array[AnyRef] = filterTypedActors(_ true)
/**
* Invokes a function for all typed actors.
*/
def foreachTypedActor(f: (AnyRef) Unit) = {
val i = typedActorsByUuid.values.iterator
while (i.hasNext)
f(i.next)
}
/**
* Invokes the function on all known typed actors until it returns Some
* Returns None if the function never returns Some
*/
def findTypedActor[T](f: PartialFunction[AnyRef, T]): Option[T] = {
val i = typedActorsByUuid.values.iterator
while (i.hasNext) {
val proxy = i.next
if (f isDefinedAt proxy) return Some(f(proxy))
}
None
}
/**
* Finds all typed actors that satisfy a predicate.
*/
def filterTypedActors(p: AnyRef Boolean): Array[AnyRef] = {
val all = new ListBuffer[AnyRef]
val i = typedActorsByUuid.values.iterator
while (i.hasNext) {
val proxy = i.next
if (p(proxy)) all += proxy
}
all.toArray
}
/**
* Get the typed actor proxy for a given typed actor ref.
*/
private def typedActorFor(actorRef: ActorRef): Option[AnyRef] =
typedActorFor(actorRef.uuid)
/**
* Registers an ClusterActorRef in the ActorRegistry.
*/
private[akka] def registerClusterActorRef(actor: ActorRef) {
val address = actor.address
clusterActorRefsByAddress.put(address, actor)
clusterActorRefsByUuid.put(actor.uuid, actor)
}
/**
* Unregisters an ClusterActorRef in the ActorRegistry.
*/
private[akka] def unregisterClusterActorRef(address: String) {
val actor = clusterActorRefsByAddress remove address
clusterActorRefsByUuid remove actor.uuid
}
/**
* Unregisters an ClusterActorRef in the ActorRegistry.
*/
private[akka] def unregisterClusterActorRef(actor: ActorRef) {
unregisterClusterActorRef(actor.address)
}
}

View file

@ -58,7 +58,9 @@ trait BootableActorLoaderService extends Bootable {
abstract override def onUnload() = {
super.onUnload()
app.registry.local.shutdownAll
// FIXME shutdown all actors
// app.registry.local.shutdownAll
}
}

View file

@ -12,6 +12,7 @@ import akka.event.EventHandler
import akka.actor.DeploymentConfig._
import akka.{ AkkaException, AkkaApplication }
import akka.config.{ Configuration, ConfigurationException }
import akka.util.Duration
trait ActorDeployer {
private[akka] def init(deployments: Seq[Deploy]): Unit
@ -127,7 +128,7 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
val addressPath = "akka.actor.deployment." + address
configuration.getSection(addressPath) match {
case None
Some(Deploy(address, None, Direct, NrOfInstances(1), RemoveConnectionOnFirstFailureLocalFailureDetector, LocalScope))
Some(Deploy(address, None, Direct, NrOfInstances(1), NoOpFailureDetector, LocalScope))
case Some(addressConfig)
@ -138,15 +139,11 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
case "direct" Direct
case "round-robin" RoundRobin
case "random" Random
case "scatter-gather" ScatterGather
case "least-cpu" LeastCPU
case "least-ram" LeastRAM
case "least-messages" LeastMessages
case customRouterClassName
createInstance[AnyRef](customRouterClassName, emptyParams, emptyArguments).fold(
e throw new ConfigurationException(
"Config option [" + addressPath + ".router] needs to be one of " +
"[\"direct\", \"round-robin\", \"random\", \"least-cpu\", \"least-ram\", \"least-messages\" or the fully qualified name of Router class]", e),
CustomRouter(_))
case routerClassName CustomRouter(routerClassName)
}
// --------------------------------
@ -174,7 +171,7 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
}
// --------------------------------
// akka.actor.deployment.<address>.failure-detector.xxx
// akka.actor.deployment.<address>.failure-detector.<detector>
// --------------------------------
val failureDetectorOption: Option[FailureDetector] = addressConfig.getSection("failure-detector") match {
case Some(failureDetectorConfig)
@ -182,22 +179,27 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
case Nil None
case detector :: Nil
detector match {
case "remove-connection-on-first-local-failure"
Some(RemoveConnectionOnFirstFailureLocalFailureDetector)
case "no-op"
Some(NoOpFailureDetector)
case "remove-connection-on-first-failure"
Some(RemoveConnectionOnFirstFailureFailureDetector)
case "bannage-period"
throw new ConfigurationException(
"Configuration for [" + addressPath + ".failure-detector.bannage-period] must have a 'time-to-ban' option defined")
case "bannage-period.time-to-ban"
failureDetectorConfig.getSection("bannage-period") map { section
BannagePeriodFailureDetector(section.getInt("time-to-ban", 10))
val timeToBan = Duration(section.getInt("time-to-ban", 60), application.AkkaConfig.DefaultTimeUnit)
BannagePeriodFailureDetector(timeToBan)
}
case "custom"
failureDetectorConfig.getSection("custom") map { section
val implementationClass = section.getString("class").getOrElse(throw new ConfigurationException(
"Configuration for [" + addressPath +
"failure-detector.custom] must have a 'class' element with the fully qualified name of the failure detector class"))
".failure-detector.custom] must have a 'class' element with the fully qualified name of the failure detector class"))
CustomFailureDetector(implementationClass)
}
@ -206,11 +208,11 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
case detectors
throw new ConfigurationException(
"Configuration for [" + addressPath +
"failure-detector] can not have multiple sections - found [" + detectors.mkString(", ") + "]")
".failure-detector] can not have multiple sections - found [" + detectors.mkString(", ") + "]")
}
case None None
}
val failureDetector = failureDetectorOption getOrElse { BannagePeriodFailureDetector(10) } // fall back to default failure detector
val failureDetector = failureDetectorOption getOrElse { NoOpFailureDetector } // fall back to default failure detector
// --------------------------------
// akka.actor.deployment.<address>.create-as
@ -267,7 +269,7 @@ class Deployer(val application: AkkaApplication) extends ActorDeployer {
// --------------------------------
addressConfig.getSection("cluster") match {
case None
Some(Deploy(address, recipe, router, nrOfInstances, RemoveConnectionOnFirstFailureLocalFailureDetector, LocalScope)) // deploy locally
Some(Deploy(address, recipe, router, nrOfInstances, NoOpFailureDetector, LocalScope)) // deploy locally
case Some(clusterConfig)

View file

@ -4,8 +4,10 @@
package akka.actor
import akka.routing.{ RouterType, FailureDetectorType }
import akka.AkkaApplication
import akka.util.Duration
import akka.routing.{ RouterType, FailureDetectorType }
import akka.routing.FailureDetectorType._
object DeploymentConfig {
@ -17,7 +19,7 @@ object DeploymentConfig {
recipe: Option[ActorRecipe],
routing: Routing = Direct,
nrOfInstances: NrOfInstances = ZeroNrOfInstances,
failureDetector: FailureDetector = RemoveConnectionOnFirstFailureLocalFailureDetector,
failureDetector: FailureDetector = NoOpFailureDetector,
scope: Scope = LocalScope) {
Address.validate(address)
}
@ -31,12 +33,13 @@ object DeploymentConfig {
// --- Routing
// --------------------------------
sealed trait Routing
case class CustomRouter(router: AnyRef) extends Routing
case class CustomRouter(routerClassName: String) extends Routing
// For Java API
case class Direct() extends Routing
case class RoundRobin() extends Routing
case class Random() extends Routing
case class ScatterGather() extends Routing
case class LeastCPU() extends Routing
case class LeastRAM() extends Routing
case class LeastMessages() extends Routing
@ -45,6 +48,7 @@ object DeploymentConfig {
case object Direct extends Routing
case object RoundRobin extends Routing
case object Random extends Routing
case object ScatterGather extends Routing
case object LeastCPU extends Routing
case object LeastRAM extends Routing
case object LeastMessages extends Routing
@ -53,15 +57,15 @@ object DeploymentConfig {
// --- FailureDetector
// --------------------------------
sealed trait FailureDetector
case class BannagePeriodFailureDetector(timeToBan: Long) extends FailureDetector
case class BannagePeriodFailureDetector(timeToBan: Duration) extends FailureDetector
case class CustomFailureDetector(className: String) extends FailureDetector
// For Java API
case class RemoveConnectionOnFirstFailureLocalFailureDetector() extends FailureDetector
case class NoOpFailureDetector() extends FailureDetector
case class RemoveConnectionOnFirstFailureFailureDetector() extends FailureDetector
// For Scala API
case object RemoveConnectionOnFirstFailureLocalFailureDetector extends FailureDetector
case object NoOpFailureDetector extends FailureDetector
case object RemoveConnectionOnFirstFailureFailureDetector extends FailureDetector
// --------------------------------
@ -166,29 +170,31 @@ object DeploymentConfig {
}
def failureDetectorTypeFor(failureDetector: FailureDetector): FailureDetectorType = failureDetector match {
case BannagePeriodFailureDetector(timeToBan) FailureDetectorType.BannagePeriodFailureDetector(timeToBan)
case RemoveConnectionOnFirstFailureLocalFailureDetector FailureDetectorType.RemoveConnectionOnFirstFailureLocalFailureDetector
case RemoveConnectionOnFirstFailureLocalFailureDetector() FailureDetectorType.RemoveConnectionOnFirstFailureLocalFailureDetector
case RemoveConnectionOnFirstFailureFailureDetector FailureDetectorType.RemoveConnectionOnFirstFailureFailureDetector
case RemoveConnectionOnFirstFailureFailureDetector() FailureDetectorType.RemoveConnectionOnFirstFailureFailureDetector
case CustomFailureDetector(implClass) FailureDetectorType.CustomFailureDetector(implClass)
case unknown throw new UnsupportedOperationException("Unknown FailureDetector [" + unknown + "]")
case NoOpFailureDetector FailureDetectorType.NoOp
case NoOpFailureDetector() FailureDetectorType.NoOp
case BannagePeriodFailureDetector(timeToBan) FailureDetectorType.BannagePeriod(timeToBan)
case RemoveConnectionOnFirstFailureFailureDetector FailureDetectorType.RemoveConnectionOnFirstFailure
case RemoveConnectionOnFirstFailureFailureDetector() FailureDetectorType.RemoveConnectionOnFirstFailure
case CustomFailureDetector(implClass) FailureDetectorType.Custom(implClass)
case unknown throw new UnsupportedOperationException("Unknown FailureDetector [" + unknown + "]")
}
def routerTypeFor(routing: Routing): RouterType = routing match {
case Direct RouterType.Direct
case Direct() RouterType.Direct
case RoundRobin RouterType.RoundRobin
case RoundRobin() RouterType.RoundRobin
case Random RouterType.Random
case Random() RouterType.Random
case LeastCPU RouterType.LeastCPU
case LeastCPU() RouterType.LeastCPU
case LeastRAM RouterType.LeastRAM
case LeastRAM() RouterType.LeastRAM
case LeastMessages RouterType.LeastMessages
case LeastMessages() RouterType.LeastMessages
case c: CustomRouter throw new UnsupportedOperationException("Unknown Router [" + c + "]")
case Direct RouterType.Direct
case Direct() RouterType.Direct
case RoundRobin RouterType.RoundRobin
case RoundRobin() RouterType.RoundRobin
case Random RouterType.Random
case Random() RouterType.Random
case ScatterGather RouterType.ScatterGather
case ScatterGather() RouterType.ScatterGather
case LeastCPU RouterType.LeastCPU
case LeastCPU() RouterType.LeastCPU
case LeastRAM RouterType.LeastRAM
case LeastRAM() RouterType.LeastRAM
case LeastMessages RouterType.LeastMessages
case LeastMessages() RouterType.LeastMessages
case CustomRouter(implClass) RouterType.Custom(implClass)
}
def isReplicated(replicationScheme: ReplicationScheme): Boolean =

View file

@ -28,13 +28,7 @@ object Scheduler {
private[akka] val service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory)
private def createSendRunnable(receiver: ActorRef, message: Any, throwWhenReceiverExpired: Boolean): Runnable = {
new Runnable {
def run =
if (receiver.isShutdown && throwWhenReceiverExpired)
throw new RuntimeException("Receiver not found, unregistered")
else
receiver ! message
}
new Runnable { def run = receiver ! message }
}
/**

View file

@ -267,8 +267,9 @@ class TypedActor(val application: AkkaApplication) {
private[akka] class TypedActor[R <: AnyRef, T <: R](val proxyVar: AtomVar[R], createInstance: T) extends Actor {
override def preStart = application.registry.registerTypedActor(self, proxyVar.get) //Make sure actor registry knows about this actor
override def postStop = application.registry.unregisterTypedActor(self, proxyVar.get)
// FIXME TypedActor register/unregister on postStop/preStart
// override def preStart = application.registry.registerTypedActor(self, proxyVar.get) //Make sure actor registry knows about this actor
// override def postStop = application.registry.unregisterTypedActor(self, proxyVar.get)
val me = createInstance
def receive = {

View file

@ -805,14 +805,27 @@ trait Promise[T] extends Future[T] {
//Companion object to FState, just to provide a cheap, immutable default entry
private[akka] object FState {
val empty = new FState[Nothing]()
def apply[T](): FState[T] = empty.asInstanceOf[FState[T]]
}
def apply[T](): FState[T] = EmptyPending.asInstanceOf[FState[T]]
/**
* Represents the internal state of the DefaultCompletableFuture
*/
private[akka] case class FState[T](value: Option[Either[Throwable, T]] = None, listeners: List[Future[T] Unit] = Nil)
/**
* Represents the internal state of the DefaultCompletableFuture
*/
sealed trait FState[+T] { def value: Option[Either[Throwable, T]] }
case class Pending[T](listeners: List[Future[T] Unit] = Nil) extends FState[T] {
def value: Option[Either[Throwable, T]] = None
}
case class Success[T](value: Option[Either[Throwable, T]] = None) extends FState[T] {
def result: T = value.get.right.get
}
case class Failure[T](value: Option[Either[Throwable, T]] = None) extends FState[T] {
def exception: Throwable = value.get.left.get
}
case object Expired extends FState[Nothing] {
def value: Option[Either[Throwable, Nothing]] = None
}
val EmptyPending = Pending[Nothing](Nil)
}
/**
* The default concrete Future implementation.
@ -820,6 +833,8 @@ private[akka] case class FState[T](value: Option[Either[Throwable, T]] = None, l
class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDispatcher) extends AbstractPromise with Promise[T] {
self
import FState.{ FState, Success, Failure, Pending, Expired }
def this()(implicit dispatcher: MessageDispatcher, timeout: Timeout) = this(timeout)
def this(timeout: Long)(implicit dispatcher: MessageDispatcher) = this(Timeout(timeout))
@ -867,7 +882,18 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
AbstractPromise.updater.asInstanceOf[AtomicReferenceFieldUpdater[AbstractPromise, FState[T]]].compareAndSet(this, oldState, newState)
@inline
protected final def getState: FState[T] = AbstractPromise.updater.asInstanceOf[AtomicReferenceFieldUpdater[AbstractPromise, FState[T]]].get(this)
protected final def getState: FState[T] = {
@tailrec
def read(): FState[T] = {
val cur = AbstractPromise.updater.asInstanceOf[AtomicReferenceFieldUpdater[AbstractPromise, FState[T]]].get(this)
if (cur.isInstanceOf[Pending[_]] && isExpired) {
if (updateState(cur, Expired)) Expired else read()
} else cur
}
read()
}
def complete(value: Either[Throwable, T]): this.type = {
val callbacks = {
@ -875,15 +901,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
@tailrec
def tryComplete: List[Future[T] Unit] = {
val cur = getState
if (cur.value.isDefined) Nil
else if ( /*cur.value.isEmpty && */ isExpired) {
//Empty and expired, so remove listeners
//TODO Perhaps cancel existing onTimeout listeners in the future here?
updateState(cur, FState()) //Try to reset the state to the default, doesn't matter if it fails
Nil
} else {
if (updateState(cur, FState(Option(value), Nil))) cur.listeners
else tryComplete
cur match {
case Pending(listeners)
if (updateState(cur, if (value.isLeft) Failure(Some(value)) else Success(Some(value)))) listeners
else tryComplete
case _ Nil
}
}
tryComplete
@ -901,10 +924,14 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
@tailrec //Returns whether the future has already been completed or not
def tryAddCallback(): Boolean = {
val cur = getState
if (cur.value.isDefined) true
else if (isExpired) false
else if (updateState(cur, cur.copy(listeners = func :: cur.listeners))) false
else tryAddCallback()
cur match {
case _: Success[_] | _: Failure[_] true
case Expired false
case p: Pending[_]
val pt = p.asInstanceOf[Pending[T]]
if (updateState(pt, pt.copy(listeners = func :: pt.listeners))) false
else tryAddCallback()
}
}
if (tryAddCallback()) Future.dispatchTask(() notifyCompleted(func))
@ -937,10 +964,10 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
final def orElse[A >: T](fallback: A): Future[A] =
if (timeout.duration.isFinite) {
value match {
case Some(_) this
case _ if isExpired Future[A](fallback, timeout)
case _
getState match {
case _: Success[_] | _: Failure[_] this
case Expired Future[A](fallback, timeout)
case _: Pending[_]
val promise = new DefaultPromise[A](Timeout.never) //TODO FIXME We can't have infinite timeout here, doesn't make sense.
promise completeWith this
val runnable = new Runnable {

View file

@ -51,14 +51,23 @@ object ThreadPoolConfig {
}
}
/**
* Function0 without the fun stuff (mostly for the sake of the Java API side of things)
*/
trait ExecutorServiceFactory {
def createExecutorService: ExecutorService
}
/**
* Generic way to specify an ExecutorService to a Dispatcher, create it with the given name if desired
*/
trait ExecutorServiceFactoryProvider {
def createExecutorServiceFactory(name: String): ExecutorServiceFactory
}
/**
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
*/
case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
@ -89,6 +98,9 @@ object ThreadPoolConfigDispatcherBuilder {
def conf_?[T](opt: Option[T])(fun: (T) ThreadPoolConfigDispatcherBuilder ThreadPoolConfigDispatcherBuilder): Option[(ThreadPoolConfigDispatcherBuilder) ThreadPoolConfigDispatcherBuilder] = opt map fun
}
/**
* A DSL to configure and create a MessageDispatcher with a ThreadPoolExecutor
*/
case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfig) MessageDispatcher, config: ThreadPoolConfig) extends DispatcherBuilder {
import ThreadPoolConfig._
def build = dispatcherFactory(config)
@ -223,6 +235,9 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend
}
}
/**
* As the name says
*/
trait ExecutorServiceDelegate extends ExecutorService {
def executor: ExecutorService
@ -254,6 +269,9 @@ trait ExecutorServiceDelegate extends ExecutorService {
def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit)
}
/**
* An ExecutorService that only creates the underlying Executor if any of the methods of the ExecutorService are called
*/
trait LazyExecutorService extends ExecutorServiceDelegate {
def createExecutor: ExecutorService
@ -263,6 +281,9 @@ trait LazyExecutorService extends ExecutorServiceDelegate {
}
}
/**
* A concrete implementation of LazyExecutorService (Scala API)
*/
class LazyExecutorServiceWrapper(executorFactory: ExecutorService) extends LazyExecutorService {
def createExecutor = executorFactory
}

View file

@ -6,116 +6,31 @@ package akka.event
import akka.actor._
trait DeathWatch {
def signal(terminated: Terminated): Unit
/**
* The contract of DeathWatch is not properly expressed using the type system
* Whenever there is a publish, all listeners to the Terminated Actor should be atomically removed
* A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down
* See InVMMonitoring for semantics
*/
trait DeathWatch extends ActorEventBus with ActorClassifier {
type Event = Terminated
protected final def classify(event: Event): Classifier = event.actor
}
trait Monitoring {
object InVMMonitoring extends DeathWatch with ActorClassification {
def link(monitor: ActorRef, monitored: ActorRef): Unit
def mapSize = 1024
def unlink(monitor: ActorRef, monitored: ActorRef): Unit
}
object InVMMonitoring extends DeathWatch with Monitoring {
class MonitoringBook(mapSize: Int = 1024) {
import java.util.concurrent.ConcurrentHashMap
import scala.annotation.tailrec
val mappings = new ConcurrentHashMap[ActorRef, Vector[ActorRef]](mapSize)
@tailrec
final def associate(monitored: ActorRef, monitor: ActorRef): Boolean = {
val current = mappings get monitored
current match {
case null
if (monitored.isShutdown) false
else {
if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor)
else {
if (monitored.isShutdown) !dissociate(monitored, monitor)
else true
}
}
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
if (monitored.isShutdown) false
if (v.contains(monitor)) true
else {
val added = v :+ monitor
if (!mappings.replace(monitored, v, added)) associate(monitored, monitor)
else {
if (monitored.isShutdown) !dissociate(monitored, monitor)
else true
}
}
}
}
final def dissociate(monitored: ActorRef): Iterable[ActorRef] = {
@tailrec
def dissociateAsMonitored(monitored: ActorRef): Iterable[ActorRef] = {
val current = mappings get monitored
current match {
case null Vector.empty[ActorRef]
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
if (!mappings.remove(monitored, v)) dissociateAsMonitored(monitored)
else v
}
}
def dissociateAsMonitor(monitor: ActorRef): Unit = {
val i = mappings.entrySet.iterator
while (i.hasNext()) {
val entry = i.next()
val v = entry.getValue
v match {
case raw: Vector[_]
val monitors = raw.asInstanceOf[Vector[ActorRef]]
if (monitors.contains(monitor))
dissociate(entry.getKey, monitor)
case _ //Dun care
}
}
}
try { dissociateAsMonitored(monitored) } finally { dissociateAsMonitor(monitored) }
}
@tailrec
final def dissociate(monitored: ActorRef, monitor: ActorRef): Boolean = {
val current = mappings get monitored
current match {
case null false
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
val removed = v.filterNot(monitor ==)
if (removed eq v) false
else if (removed.isEmpty) {
if (!mappings.remove(monitored, v)) dissociate(monitored, monitor)
else true
} else {
if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor)
else true
}
}
}
override def publish(event: Event): Unit = {
val monitors = dissociate(classify(event))
if (monitors.nonEmpty) monitors.foreach(_ ! event)
}
val monitoring = new MonitoringBook(1024) //Key == monitored, Values == monitors
def signal(terminated: Terminated): Unit = {
val monitors = monitoring.dissociate(terminated.actor)
if (monitors.nonEmpty) monitors.foreach(_ ! terminated)
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = {
if (!super.subscribe(subscriber, to)) {
subscriber ! Terminated(subscriber, new ActorKilledException("Already terminated when linking"))
false
} else true
}
def link(monitor: ActorRef, monitored: ActorRef): Unit = {
if (!monitoring.associate(monitored, monitor))
monitor ! Terminated(monitored, new ActorKilledException("Already terminated when linking"))
}
def unlink(monitor: ActorRef, monitored: ActorRef): Unit =
monitoring.dissociate(monitored, monitor)
}

View file

@ -0,0 +1,275 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.event
import akka.actor.ActorRef
import akka.util.Index
import java.util.concurrent.ConcurrentSkipListSet
import java.util.Comparator
/**
* Represents the base type for EventBuses
* Internally has an Event type, a Classifier type and a Subscriber type
*
* For the Java API, @see akka.event.japi.*
*/
trait EventBus {
type Event
type Classifier
type Subscriber
/**
* Attempts to register the subscriber to the specified Classifier
* @returns true if successful and false if not (because it was already subscribed to that Classifier, or otherwise)
*/
def subscribe(subscriber: Subscriber, to: Classifier): Boolean
/**
* Attempts to deregister the subscriber from the specified Classifier
* @returns true if successful and false if not (because it wasn't subscribed to that Classifier, or otherwise)
*/
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean
/**
* Attempts to deregister the subscriber from all Classifiers it may be subscribed to
*/
def unsubscribe(subscriber: Subscriber): Unit
/**
* Publishes the specified Event to this bus
*/
def publish(event: Event): Unit
}
/**
* Represents an EventBus where the Subscriber type is ActorRef
*/
trait ActorEventBus extends EventBus {
type Subscriber = ActorRef
}
/**
* Can be mixed into an EventBus to specify that the Classifier type is ActorRef
*/
trait ActorClassifier { self: EventBus
type Classifier = ActorRef
}
/**
* Can be mixed into an EventBus to specify that the Classifier type is a Function from Event to Boolean (predicate)
*/
trait PredicateClassifier { self: EventBus
type Classifier = Event Boolean
}
/**
* Maps Subscribers to Classifiers using equality on Classifier to store a Set of Subscribers (hence the need for compareSubscribers)
* Maps Events to Classifiers through the classify-method (so it knows who to publish to)
*
* The compareSubscribers need to provide a total ordering of the Subscribers
*/
trait LookupClassification { self: EventBus
protected final val subscribers = new Index[Classifier, Subscriber](mapSize(), new Comparator[Subscriber] {
def compare(a: Subscriber, b: Subscriber): Int = compareSubscribers(a, b)
})
/**
* This is a size hint for the number of Classifiers you expect to have (use powers of 2)
*/
protected def mapSize(): Int
/**
* Provides a total ordering of Subscribers (think java.util.Comparator.compare)
*/
protected def compareSubscribers(a: Subscriber, b: Subscriber): Int
/**
* Returns the Classifier associated with the given Event
*/
protected def classify(event: Event): Classifier
/**
* Publishes the given Event to the given Subscriber
*/
protected def publish(event: Event, subscriber: Subscriber): Unit
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = subscribers.put(to, subscriber)
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = subscribers.remove(from, subscriber)
def unsubscribe(subscriber: Subscriber): Unit = subscribers.removeValue(subscriber)
def publish(event: Event): Unit = {
val i = subscribers.valueIterator(classify(event))
while (i.hasNext) publish(event, i.next())
}
}
/**
* Maps Classifiers to Subscribers and selects which Subscriber should receive which publication through scanning through all Subscribers
* through the matches(classifier, event) method
*
* Note: the compareClassifiers and compareSubscribers must together form an absolute ordering (think java.util.Comparator.compare)
*/
trait ScanningClassification { self: EventBus
protected final val subscribers = new ConcurrentSkipListSet[(Classifier, Subscriber)](new Comparator[(Classifier, Subscriber)] {
def compare(a: (Classifier, Subscriber), b: (Classifier, Subscriber)): Int = {
val cM = compareClassifiers(a._1, b._1)
if (cM != 0) cM
else compareSubscribers(a._2, b._2)
}
})
/**
* Provides a total ordering of Classifiers (think java.util.Comparator.compare)
*/
protected def compareClassifiers(a: Classifier, b: Classifier): Int
/**
* Provides a total ordering of Subscribers (think java.util.Comparator.compare)
*/
protected def compareSubscribers(a: Subscriber, b: Subscriber): Int
/**
* Returns whether the specified Classifier matches the specified Event
*/
protected def matches(classifier: Classifier, event: Event): Boolean
/**
* Publishes the specified Event to the specified Subscriber
*/
protected def publish(event: Event, subscriber: Subscriber): Unit
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = subscribers.add((to, subscriber))
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = subscribers.remove((from, subscriber))
def unsubscribe(subscriber: Subscriber): Unit = {
val i = subscribers.iterator()
while (i.hasNext) {
val e = i.next()
if (compareSubscribers(subscriber, e._2) == 0) i.remove()
}
}
def publish(event: Event): Unit = {
val currentSubscribers = subscribers.iterator()
while (currentSubscribers.hasNext) {
val (classifier, subscriber) = currentSubscribers.next()
if (matches(classifier, event))
publish(event, subscriber)
}
}
}
/**
* Maps ActorRefs to ActorRefs to form an EventBus where ActorRefs can listen to other ActorRefs
*/
trait ActorClassification { self: ActorEventBus with ActorClassifier
import java.util.concurrent.ConcurrentHashMap
import scala.annotation.tailrec
protected val mappings = new ConcurrentHashMap[ActorRef, Vector[ActorRef]](mapSize)
@tailrec
protected final def associate(monitored: ActorRef, monitor: ActorRef): Boolean = {
val current = mappings get monitored
current match {
case null
if (monitored.isShutdown) false
else {
if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor)
else {
if (monitored.isShutdown) !dissociate(monitored, monitor)
else true
}
}
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
if (monitored.isShutdown) false
if (v.contains(monitor)) true
else {
val added = v :+ monitor
if (!mappings.replace(monitored, v, added)) associate(monitored, monitor)
else {
if (monitored.isShutdown) !dissociate(monitored, monitor)
else true
}
}
}
}
protected final def dissociate(monitored: ActorRef): Iterable[ActorRef] = {
@tailrec
def dissociateAsMonitored(monitored: ActorRef): Iterable[ActorRef] = {
val current = mappings get monitored
current match {
case null Vector.empty[ActorRef]
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
if (!mappings.remove(monitored, v)) dissociateAsMonitored(monitored)
else v
}
}
def dissociateAsMonitor(monitor: ActorRef): Unit = {
val i = mappings.entrySet.iterator
while (i.hasNext()) {
val entry = i.next()
val v = entry.getValue
v match {
case raw: Vector[_]
val monitors = raw.asInstanceOf[Vector[ActorRef]]
if (monitors.contains(monitor))
dissociate(entry.getKey, monitor)
case _ //Dun care
}
}
}
try { dissociateAsMonitored(monitored) } finally { dissociateAsMonitor(monitored) }
}
@tailrec
protected final def dissociate(monitored: ActorRef, monitor: ActorRef): Boolean = {
val current = mappings get monitored
current match {
case null false
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
val removed = v.filterNot(monitor ==)
if (removed eq v) false
else if (removed.isEmpty) {
if (!mappings.remove(monitored, v)) dissociate(monitored, monitor)
else true
} else {
if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor)
else true
}
}
}
/**
* Returns the Classifier associated with the specified Event
*/
protected def classify(event: Event): Classifier
/**
* This is a size hint for the number of Classifiers you expect to have (use powers of 2)
*/
protected def mapSize: Int
def publish(event: Event): Unit = mappings.get(classify(event)) match {
case null
case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]]
v foreach { _ ! event }
}
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber)
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = dissociate(from, subscriber)
def unsubscribe(subscriber: Subscriber): Unit = dissociate(subscriber)
}

View file

@ -0,0 +1,38 @@
package akka.event.japi
import akka.event._
/**
* See documentation for akka.event.LookupClassification
* E is the Event type
* S is the Subscriber type
* C is the Classifier type
*/
abstract class LookupEventBus[E, S, C] extends EventBus with LookupClassification {
type Event = E
type Subscriber = S
type Classifier = C
}
/**
* See documentation for akka.event.ScanningClassification
* E is the Event type
* S is the Subscriber type
* C is the Classifier type
*/
abstract class ScanningEventBus[E, S, C] extends EventBus with ScanningClassification {
type Event = E
type Subscriber = S
type Classifier = C
}
/**
* See documentation for akka.event.ActorClassification
* An EventBus where the Subscribers are ActorRefs and the Classifier is ActorRef
* Means that ActorRefs "listen" to other ActorRefs
* E is the Event type
*/
abstract class ActorEventBus[E] extends akka.event.ActorEventBus with ActorClassification with ActorClassifier {
}

View file

@ -0,0 +1,120 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.routing
import akka.actor._
import scala.annotation.tailrec
import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger }
import java.net.InetSocketAddress
/**
* An Iterable that also contains a version.
*/
trait VersionedIterable[A] {
val version: Long
def iterable: Iterable[A]
def apply(): Iterable[A] = iterable
}
/**
* Manages connections (ActorRefs) for a router.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait ConnectionManager {
/**
* A version that is useful to see if there is any change in the connections. If there is a change, a router is
* able to update its internal datastructures.
*/
def version: Long
/**
* Returns the number of 'available' 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
/**
* Shuts the connection manager down, which stops all managed actors
*/
def shutdown()
/**
* 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 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 connections: VersionedIterable[ActorRef]
/**
* Removes a connection from the connection manager.
*
* @param ref the dead
*/
def remove(deadRef: ActorRef)
/**
* Creates a new connection (ActorRef) if it didn't exist. Atomically.
*/
def putIfAbsent(address: InetSocketAddress, newConnectionFactory: () ActorRef): ActorRef
/**
* Fails over connections from one address to another.
*/
def failOver(from: InetSocketAddress, to: InetSocketAddress)
}
/**
* Manages local connections for a router, e.g. local actors.
*/
class LocalConnectionManager(initialConnections: Iterable[ActorRef]) extends ConnectionManager {
case class State(version: Long, connections: Iterable[ActorRef]) extends VersionedIterable[ActorRef] {
def iterable = connections
}
private val state: AtomicReference[State] = new AtomicReference[State](newState())
private def newState() = State(Long.MinValue, initialConnections)
def version: Long = state.get.version
def size: Int = state.get.connections.size
def connections = state.get
def shutdown() {
state.get.connections foreach (_.stop())
}
@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)
if (newList.size != oldState.connections.size) {
//one or more occurrences of the actorRef were removed, so we need to update the state.
val newState = State(oldState.version + 1, newList)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) remove(ref)
}
}
def failOver(from: InetSocketAddress, to: InetSocketAddress) {} // do nothing here
def putIfAbsent(address: InetSocketAddress, newConnectionFactory: () ActorRef): ActorRef = {
throw new UnsupportedOperationException("Not supported")
}
}

View file

@ -1,148 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.routing
import akka.AkkaException
import akka.actor._
import akka.event.EventHandler
import akka.config.ConfigurationException
import akka.actor.UntypedChannel._
import akka.dispatch.Future
import akka.util.ReflectiveAccess
import java.net.InetSocketAddress
import java.lang.reflect.InvocationTargetException
import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger }
import scala.annotation.tailrec
sealed trait FailureDetectorType
/**
* Used for declarative configuration of failure detection.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object FailureDetectorType {
case object RemoveConnectionOnFirstFailureLocalFailureDetector extends FailureDetectorType
case object RemoveConnectionOnFirstFailureFailureDetector extends FailureDetectorType
case class BannagePeriodFailureDetector(timeToBan: Long) extends FailureDetectorType
case class CustomFailureDetector(className: String) extends FailureDetectorType
}
/**
* Misc helper and factory methods for failure detection.
*/
object FailureDetector {
def createCustomFailureDetector(
implClass: String,
connections: Map[InetSocketAddress, ActorRef]): FailureDetector = {
ReflectiveAccess.createInstance(
implClass,
Array[Class[_]](classOf[Map[InetSocketAddress, ActorRef]]),
Array[AnyRef](connections)) match {
case Right(actor) actor
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
throw new ConfigurationException(
"Could not instantiate custom FailureDetector of [" +
implClass + "] due to: " +
cause, cause)
}
}
}
/**
* The FailureDetector acts like a middleman between the Router and
* the actor reference that does the routing and can dectect and act upon failure.
*
* Through the FailureDetector:
* <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>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait FailureDetector {
/**
* Returns true if the 'connection' is considered available.
*/
def isAvailable(connection: InetSocketAddress): Boolean
/**
* Records a successful connection.
*/
def recordSuccess(connection: InetSocketAddress, timestamp: Long)
/**
* Records a failed connection.
*/
def recordFailure(connection: InetSocketAddress, timestamp: Long)
/**
* A version that is useful to see if there is any change in the connections. If there is a change, a router is
* able to update its internal datastructures.
*/
def version: Long
/**
* 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
/**
* Stops all managed actors
*/
def stopAll()
/**
* 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 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 versionedIterable: VersionedIterable[ActorRef]
/**
* A callback that can be used to indicate that a connected actorRef was dead.
* <p/>
* Implementations should make sure that this method can be called without the actorRef being part of the
* current set of connections. The most logical way to deal with this situation, is just to ignore it. One of the
* reasons this can happen is that multiple thread could at the 'same' moment discover for the same ActorRef that
* not working.
*
* 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.
*
* @param ref the dead
*/
def remove(deadRef: ActorRef)
/**
* TODO: document
*/
def putIfAbsent(address: InetSocketAddress, newConnectionFactory: () ActorRef): ActorRef
/**
* Fails over connections from one address to another.
*/
def failOver(from: InetSocketAddress, to: InetSocketAddress)
}

View file

@ -6,11 +6,25 @@ package akka.routing
import akka.util.Duration
import akka.actor._
import akka.util.ReflectiveAccess
import akka.util.{ ReflectiveAccess, Duration }
import java.net.InetSocketAddress
import scala.collection.JavaConversions.iterableAsScalaIterable
import scala.collection.JavaConversions.{ iterableAsScalaIterable, mapAsScalaMap }
sealed trait FailureDetectorType
/**
* Used for declarative configuration of failure detection.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object FailureDetectorType {
case object NoOp extends FailureDetectorType
case object RemoveConnectionOnFirstFailure extends FailureDetectorType
case class BannagePeriod(timeToBan: Duration) extends FailureDetectorType
case class Custom(className: String) extends FailureDetectorType
}
sealed trait RouterType
@ -33,6 +47,11 @@ object RouterType {
*/
object RoundRobin extends RouterType
/**
* A RouterType that selects the connection by using scatter gather.
*/
object ScatterGather extends RouterType
/**
* A RouterType that selects the connection based on the least amount of cpu usage
*/
@ -53,25 +72,10 @@ object RouterType {
/**
* A user-defined custom RouterType.
*/
object Custom extends RouterType
case class Custom(implClass: String) extends RouterType
}
object RoutedProps {
final val defaultTimeout = Timeout(Duration.MinusInf)
final val defaultRouterFactory = () new RoundRobinRouter
final val defaultLocalOnly = false
final val defaultFailureDetectorFactory = (connections: Map[InetSocketAddress, ActorRef]) new RemoveConnectionOnFirstFailureLocalFailureDetector(connections.values)
/**
* 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.
*
@ -86,12 +90,11 @@ object RoutedProps {
*/
case class RoutedProps(
routerFactory: () Router,
connections: Iterable[ActorRef],
failureDetectorFactory: (Map[InetSocketAddress, ActorRef]) FailureDetector = RoutedProps.defaultFailureDetectorFactory,
connectionManager: ConnectionManager,
timeout: Timeout = RoutedProps.defaultTimeout,
localOnly: Boolean = RoutedProps.defaultLocalOnly) {
def this() = this(RoutedProps.defaultRouterFactory, List())
def this() = this(RoutedProps.defaultRouterFactory, new LocalConnectionManager(List()))
/**
* Returns a new RoutedProps configured with a random router.
@ -150,28 +153,35 @@ case class RoutedProps(
*
* Scala API.
*/
def withConnections(c: Iterable[ActorRef]): RoutedProps = copy(connections = c)
def withLocalConnections(c: Iterable[ActorRef]): RoutedProps = copy(connectionManager = new LocalConnectionManager(c))
/**
* Sets the connections to use.
*
* Java API.
*/
def withConnections(c: java.lang.Iterable[ActorRef]): RoutedProps = copy(connections = iterableAsScalaIterable(c))
def withLocalConnections(c: java.lang.Iterable[ActorRef]): RoutedProps = copy(connectionManager = new LocalConnectionManager(iterableAsScalaIterable(c)))
/**
* Returns a new RoutedProps configured with a FailureDetector factory.
* Sets the connections to use.
*
* Scala API.
*/
def withFailureDetector(failureDetectorFactory: (Map[InetSocketAddress, ActorRef]) FailureDetector): RoutedProps =
copy(failureDetectorFactory = failureDetectorFactory)
// def withRemoteConnections(c: Map[InetSocketAddress, ActorRef]): RoutedProps = copy(connectionManager = new RemoteConnectionManager(c))
/**
* Returns a new RoutedProps configured with a FailureDetector factory.
* Sets the connections to use.
*
* Java API.
*/
def withFailureDetector(failureDetectorFactory: akka.japi.Function[Map[InetSocketAddress, ActorRef], FailureDetector]): RoutedProps =
copy(failureDetectorFactory = (connections: Map[InetSocketAddress, ActorRef]) failureDetectorFactory.apply(connections))
// def withRemoteConnections(c: java.util.collection.Map[InetSocketAddress, ActorRef]): RoutedProps = copy(connectionManager = new RemoteConnectionManager(mapAsScalaMap(c)))
}
object RoutedProps {
final val defaultTimeout = Timeout(Duration.MinusInf)
final val defaultRouterFactory = () new RoundRobinRouter
final val defaultLocalOnly = false
def apply() = new RoutedProps()
}

View file

@ -10,7 +10,7 @@ import akka.event.EventHandler
import akka.config.ConfigurationException
import akka.dispatch.{ Future, MessageDispatcher }
import akka.AkkaApplication
import akka.util.ReflectiveAccess
import java.net.InetSocketAddress
import java.lang.reflect.InvocationTargetException
import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger }
@ -35,7 +35,7 @@ trait Router {
* JMM Guarantees:
* This method guarantees that all changes made in this method, are visible before one of the routing methods is called.
*/
def init(connections: FailureDetector)
def init(connectionManager: ConnectionManager)
/**
* Routes the message to one of the connections.
@ -53,150 +53,40 @@ trait Router {
def route[T](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[T]
}
/**
* An Iterable that also contains a version.
*/
trait VersionedIterable[A] {
val version: Long
def iterable: Iterable[A]
def apply(): Iterable[A] = iterable
}
/**
* An {@link AkkaException} thrown when something goes wrong while routing a message
*/
class RoutingException(message: String) extends AkkaException(message)
/**
* Default "local" failure detector. This failure detector removes an actor from the
* router if an exception occured in the router's thread (e.g. when trying to add
* the message to the receiver's mailbox).
* A Helper class to create actor references that use routing.
*/
class RemoveConnectionOnFirstFailureLocalFailureDetector extends FailureDetector {
case class State(version: Long, iterable: Iterable[ActorRef]) extends VersionedIterable[ActorRef]
private val state = new AtomicReference[State]
def this(connectionIterable: Iterable[ActorRef]) = {
this()
state.set(State(Long.MinValue, connectionIterable))
}
def isAvailable(connection: InetSocketAddress): Boolean =
state.get.iterable.find(c connection == c).isDefined
def recordSuccess(connection: InetSocketAddress, timestamp: Long) {}
def recordFailure(connection: InetSocketAddress, timestamp: Long) {}
def version: Long = state.get.version
def size: Int = state.get.iterable.size
def versionedIterable = state.get
def stopAll() {
state.get.iterable foreach (_.stop())
}
@tailrec
final def remove(ref: ActorRef) = {
val oldState = state.get
//remote the ref from the connections.
var newList = oldState.iterable.filter(currentActorRef currentActorRef ne ref)
if (newList.size != oldState.iterable.size) {
//one or more occurrences of the actorRef were removed, so we need to update the state.
val newState = State(oldState.version + 1, newList)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) remove(ref)
}
}
def failOver(from: InetSocketAddress, to: InetSocketAddress) {} // do nothing here
def putIfAbsent(address: InetSocketAddress, newConnectionFactory: () ActorRef): ActorRef = {
throw new UnsupportedOperationException("Not supported")
}
}
object Routing {
sealed trait RoutingMessage
/**
* Used to broadcast a message to all connections in a router. E.g. every connection gets the message
* regardless of their routing algorithm.
*/
case class Broadcast(message: Any) extends RoutingMessage
}
/**
* A Helper class to create actor references that use routing.
*/
class Routing(val application: AkkaApplication) {
/**
* FIXME: will very likely be moved to the ActorRef.
*/
def actorOf(props: RoutedProps, address: String = newUuid().toString): ActorRef = {
//TODO Implement support for configuring by deployment ID etc
//TODO If address matches an already created actor (Ahead-of-time deployed) return that actor
//TODO If address 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 = application.reflective.ClusterModule.isEnabled
if (clusteringEnabled && !props.localOnly)
application.reflective.ClusterModule.newClusteredActorRef(props)
else {
if (props.connections.isEmpty) //FIXME Shouldn't this be checked when instance is created so that it works with linking instead of barfing?
throw new IllegalArgumentException("A routed actorRef can't have an empty connection set")
new RoutedActorRef(props, address)
}
}
/**
* Creates a new started RoutedActorRef that uses routing to deliver a message to one of its connected actors.
*
* @param actorAddress the address of the ActorRef.
* @param connections an Iterable pointing to all connected actor references.
* @param routerType the type of routing that should be used.
* @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("Use 'Routing.actorOf(props: RoutedProps)' instead.", "2.0")
def actorOf(actorAddress: String, connections: Iterable[ActorRef], routerType: RouterType): ActorRef = {
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
new DirectRouter
case RouterType.Random
new RandomRouter
case RouterType.RoundRobin
new RoundRobinRouter
case r
throw new IllegalArgumentException("Unsupported routerType " + r)
}
if (connections.size == 0)
throw new IllegalArgumentException("To create a routed actor ref, at least one connection is required")
new RoutedActorRef(
new RoutedProps(
() router,
connections,
RoutedProps.defaultFailureDetectorFactory,
RoutedProps.defaultTimeout, true),
actorAddress)
def createCustomRouter(implClass: String): Router = {
ReflectiveAccess.createInstance(
implClass,
Array[Class[_]](),
Array[AnyRef]()) match {
case Right(router) router.asInstanceOf[Router]
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
throw new ConfigurationException(
"Could not instantiate custom Router of [" +
implClass + "] due to: " +
cause, cause)
}
}
}
@ -245,7 +135,7 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps, val address: St
}
}
router.init(new RemoveConnectionOnFirstFailureLocalFailureDetector(routedProps.connections))
router.init(routedProps.connectionManager)
}
/**
@ -257,21 +147,21 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps, val address: St
trait BasicRouter extends Router {
@volatile
protected var connections: FailureDetector = _
protected var connectionManager: ConnectionManager = _
def init(connections: FailureDetector) = {
this.connections = connections
def init(connectionManager: ConnectionManager) = {
this.connectionManager = connectionManager
}
def route(message: Any)(implicit sender: Option[ActorRef]) = message match {
case Routing.Broadcast(message)
//it is a broadcast message, we are going to send to message to all connections.
connections.versionedIterable.iterable foreach { connection
connectionManager.connections.iterable foreach { connection
try {
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
} catch {
case e: Exception
connections.remove(connection)
connectionManager.remove(connection)
throw e
}
}
@ -283,7 +173,7 @@ trait BasicRouter extends Router {
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
} catch {
case e: Exception
connections.remove(connection)
connectionManager.remove(connection)
throw e
}
case None
@ -303,7 +193,7 @@ trait BasicRouter extends Router {
connection.?(message, timeout)(sender).asInstanceOf[Future[T]]
} catch {
case e: Exception
connections.remove(connection)
connectionManager.remove(connection)
throw e
}
case None
@ -330,33 +220,32 @@ class DirectRouter extends BasicRouter {
private val state = new AtomicReference[DirectRouterState]
lazy val next: Option[ActorRef] = {
val currentState = getState
if (currentState.ref == null) None else Some(currentState.ref)
val current = currentState
if (current.ref == null) None else Some(current.ref)
}
// FIXME rename all 'getState' methods to 'currentState', non-scala
@tailrec
private def getState: DirectRouterState = {
val currentState = state.get
private def currentState: DirectRouterState = {
val current = state.get
if (currentState != null && connections.version == currentState.version) {
if (current != null && connectionManager.version == current.version) {
//we are lucky since nothing has changed in the connections.
currentState
current
} 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 versionedIterable = connections.versionedIterable
val connections = connectionManager.connections
val connectionCount = versionedIterable.iterable.size
val connectionCount = connections.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(versionedIterable.iterable.head, versionedIterable.version)
if (state.compareAndSet(currentState, newState))
val newState = new DirectRouterState(connections.iterable.head, connections.version)
if (state.compareAndSet(current, 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.
getState
currentState // recur
}
}
@ -375,28 +264,28 @@ class RandomRouter extends BasicRouter {
//FIXME: threadlocal random?
private val random = new java.util.Random(System.nanoTime)
def next: Option[ActorRef] = getState.array match {
def next: Option[ActorRef] = currentState.array match {
case a if a.isEmpty None
case a Some(a(random.nextInt(a.length)))
}
@tailrec
private def getState: RandomRouterState = {
val currentState = state.get
private def currentState: RandomRouterState = {
val current = state.get
if (currentState != null && currentState.version == connections.version) {
if (current != null && current.version == connectionManager.version) {
//we are lucky, since there has not been any change in the connections. So therefor we can use the existing state.
currentState
current
} else {
//there has been a change in connections, or it was the first try, so we need to update the internal state
val versionedIterable = connections.versionedIterable
val newState = new RandomRouterState(versionedIterable.iterable.toIndexedSeq, versionedIterable.version)
if (state.compareAndSet(currentState, newState))
val connections = connectionManager.connections
val newState = new RandomRouterState(connections.iterable.toIndexedSeq, connections.version)
if (state.compareAndSet(current, 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.
getState
currentState
}
}
@ -412,25 +301,25 @@ class RoundRobinRouter extends BasicRouter {
private val state = new AtomicReference[RoundRobinState]
def next: Option[ActorRef] = getState.next
def next: Option[ActorRef] = currentState.next
@tailrec
private def getState: RoundRobinState = {
val currentState = state.get
private def currentState: RoundRobinState = {
val current = state.get
if (currentState != null && currentState.version == connections.version) {
if (current != null && current.version == connectionManager.version) {
//we are lucky, since there has not been any change in the connections. So therefor we can use the existing state.
currentState
current
} else {
//there has been a change in connections, or it was the first try, so we need to update the internal state
val versionedIterable = connections.versionedIterable
val newState = new RoundRobinState(versionedIterable.iterable.toIndexedSeq[ActorRef], versionedIterable.version)
if (state.compareAndSet(currentState, newState))
val connections = connectionManager.connections
val newState = new RoundRobinState(connections.iterable.toIndexedSeq[ActorRef], connections.version)
if (state.compareAndSet(current, 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.
getState
currentState
}
}
@ -464,19 +353,20 @@ class RoundRobinRouter extends BasicRouter {
trait ScatterGatherRouter extends BasicRouter with Serializable {
/**
* Aggregates the responses into a single Future
* Aggregates the responses into a single Future.
*
* @param results Futures of the responses from connections
*/
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.versionedIterable.iterable.flatMap { actor
val responses = connectionManager.connections.iterable.flatMap { actor
try {
if (actor.isShutdown) throw new ActorInitializationException("For compatability - check death first")
Some(actor.?(message, timeout)(sender).asInstanceOf[Future[S]])
} catch {
case e: Exception
connections.remove(actor)
connectionManager.remove(actor)
None
}
}

View file

@ -36,7 +36,7 @@ object Crypt {
def hexify(bytes: Array[Byte]): String = {
val builder = new StringBuilder
bytes.foreach { byte builder.append(hex.charAt((byte & 0xF) >> 4)).append(hex.charAt(byte & 0xF)) }
bytes.foreach { byte builder.append(hex.charAt((byte & 0xF0) >> 4)).append(hex.charAt(byte & 0xF)) }
builder.toString
}

View file

@ -6,8 +6,8 @@ package akka.util
import annotation.tailrec
import java.util.{ Set JSet }
import java.util.concurrent.{ ConcurrentSkipListSet, ConcurrentHashMap }
import java.util.{ Comparator, Set JSet }
/**
* An implementation of a ConcurrentMultiMap
@ -16,9 +16,13 @@ import java.util.concurrent.{ ConcurrentSkipListSet, ConcurrentHashMap }
*
* @author Viktor Klang
*/
class Index[K <: AnyRef, V <: AnyRef: Manifest] {
private val Naught = Array[V]() //Nil for Arrays
private val container = new ConcurrentHashMap[K, JSet[V]]
class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
def this(mapSize: Int, cmp: (V, V) Int) = this(mapSize, new Comparator[V] {
def compare(a: V, b: V): Int = cmp(a, b)
})
private val container = new ConcurrentHashMap[K, JSet[V]](mapSize)
private val emptySet = new ConcurrentSkipListSet[V]
/**
@ -42,7 +46,7 @@ class Index[K <: AnyRef, V <: AnyRef: Manifest] {
}
}
} else {
val newSet = new ConcurrentSkipListSet[V]
val newSet = new ConcurrentSkipListSet[V](valueComparator)
newSet add v
// Parry for two simultaneous putIfAbsent(id,newSet)
@ -65,15 +69,6 @@ class Index[K <: AnyRef, V <: AnyRef: Manifest] {
spinPut(key, value)
}
/**
* @return a _new_ array of all existing values for the given key at the time of the call
*/
def values(key: K): Array[V] = {
val set: JSet[V] = container get key
val result = if (set ne null) set toArray Naught else Naught
result.asInstanceOf[Array[V]]
}
/**
* @return Some(value) for the first matching value where the supplied function returns true for the given key,
* if no matches it returns None
@ -85,6 +80,16 @@ class Index[K <: AnyRef, V <: AnyRef: Manifest] {
else None
}
/**
* Returns an Iterator of V containing the values for the supplied key, or an empty iterator if the key doesn't exist
*/
def valueIterator(key: K): scala.Iterator[V] = {
container.get(key) match {
case null Iterator.empty
case some scala.collection.JavaConversions.asScalaIterator(some.iterator())
}
}
/**
* Applies the supplied function to all keys and their values
*/
@ -112,6 +117,10 @@ class Index[K <: AnyRef, V <: AnyRef: Manifest] {
} else false //Remove failed
}
/**
* Disassociates all the values for the specified key
* @returns None if the key wasn't associated at all, or Some(scala.Iterable[V]) if it was associated
*/
def remove(key: K): Option[Iterable[V]] = {
val set = container get key
@ -123,6 +132,26 @@ class Index[K <: AnyRef, V <: AnyRef: Manifest] {
} else None //Remove failed
}
/**
* Removes the specified value from all keys
*/
def removeValue(value: V): Unit = {
val i = container.entrySet().iterator()
while (i.hasNext) {
val e = i.next()
val set = e.getValue()
if (set ne null) {
set.synchronized {
if (set.remove(value)) { //If we can remove the value
if (set.isEmpty) //and the set becomes empty
container.remove(e.getKey, emptySet) //We try to remove the key if it's mapped to an empty set
}
}
}
}
}
/**
* @return true if the underlying containers is empty, may report false negatives when the last remove is underway
*/
@ -131,5 +160,21 @@ class Index[K <: AnyRef, V <: AnyRef: Manifest] {
/**
* Removes all keys and all values
*/
def clear = foreach { case (k, v) remove(k, v) }
def clear(): Unit = {
val i = container.entrySet().iterator()
while (i.hasNext) {
val e = i.next()
val set = e.getValue()
if (set ne null) { set.synchronized { set.clear(); container.remove(e.getKey, emptySet) } }
}
}
}
/**
* An implementation of a ConcurrentMultiMap
* Adds/remove is serialized over the specified key
* Reads are fully concurrent <-- el-cheapo
*
* @author Viktor Klang
*/
class ConcurrentMultiMap[K, V](mapSize: Int, valueComparator: Comparator[V]) extends Index[K, V](mapSize, valueComparator)

View file

@ -1370,7 +1370,7 @@ class DefaultClusterNode private[akka] (
private[cluster] def failOverClusterActorRefConnections(from: InetSocketAddress, to: InetSocketAddress) {
EventHandler.info(this, "Failing over ClusterActorRef from %s to %s".format(from, to))
clusterActorRefs.values(from) foreach (_.failOver(from, to))
clusterActorRefs.valueIterator(from) foreach (_.failOver(from, to))
}
private[cluster] def migrateActorsOnFailedNodes(

View file

@ -48,17 +48,22 @@ htmlhelp_basename = 'Akkadoc'
# -- Options for LaTeX output --------------------------------------------------
def setup(app):
from sphinx.util.texescape import tex_replacements
tex_replacements.append((u'', ur'\(\Rightarrow\)'))
latex_paper_size = 'a4'
latex_font_size = '10pt'
latex_documents = [
('index', 'Akka.tex', u' Akka Documentation',
u'Scalable Solutions AB', 'manual'),
u'Typesafe Inc', 'manual'),
]
latex_elements = {
'classoptions': ',oneside,openany',
'babel': '\\usepackage[english]{babel}',
'fontpkg': '\\PassOptionsToPackage{warn}{textcomp} \\usepackage{times}',
'preamble': '\\definecolor{VerbatimColor}{rgb}{0.935,0.935,0.935}'
}

View file

@ -65,7 +65,7 @@ object Pi extends App {
val workers = Vector.fill(nrOfWorkers)(actorOf[Worker])
// wrap them with a load-balancing router
val router = Routing.actorOf(RoutedProps().withRoundRobinRouter.withConnections(workers), "pi")
val router = Routing.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
loadBalancerActor(CyclicIterator(workers))
//#create-workers

View file

@ -71,7 +71,7 @@ object Pi extends App {
val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker])
// wrap them with a load-balancing router
val router = app.routing.actorOf(RoutedProps().withRoundRobinRouter.withConnections(workers), "pi")
val router = app.createActor(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
//#create-workers
//#master-receive

View file

@ -127,7 +127,7 @@ trait RootEndpointLocator {
def configureRoot(address: String) {
def findRoot(address: String): ActorRef =
application.registry.actorFor(address).getOrElse(
application.provider.actorFor(address).getOrElse(
throw new ConfigurationException("akka.http.root-actor-id configuration option does not have a valid actor address [" + address + "]"))
root = if ((address eq null) || address == "") findRoot(application.MistSettings.RootActorID) else findRoot(address)

View file

@ -0,0 +1,180 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import scala.collection.immutable.Map
import scala.annotation.tailrec
/**
* Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper:
* [http://ddg.jaist.ac.jp/pub/HDY+04.pdf]
* <p/>
* A low threshold is prone to generate many wrong suspicions but ensures a quick detection in the event
* of a real crash. Conversely, a high threshold generates fewer mistakes but needs more time to detect
* actual crashes
* <p/>
* For example a threshold of:
* - 1 => 10% error rate
* - 2 => 1% error rate
* - 3 => 0.1% error rate -
* <p/>
* This means that for example a threshold of 3 => no heartbeat for > 6 seconds => node marked as dead/not available.
* <p/>
* Default threshold is 8 (taken from Cassandra defaults), but can be configured in the Akka config.
*/
class AccrualFailureDetector(
val threshold: Int = 8, // FIXME make these configurable
val maxSampleSize: Int = 1000) extends FailureDetector {
private case class FailureStats(mean: Double = 0.0D, variance: Double = 0.0D, deviation: Double = 0.0D)
// Implement using optimistic lockless concurrency, all state is represented
// by this immutable case class and managed by an AtomicReference
private case class State(
version: Long = 0L,
failureStats: Map[InetSocketAddress, FailureStats] = Map.empty[InetSocketAddress, FailureStats],
intervalHistory: Map[InetSocketAddress, Vector[Long]] = Map.empty[InetSocketAddress, Vector[Long]],
timestamps: Map[InetSocketAddress, Long] = Map.empty[InetSocketAddress, Long])
private val state = new AtomicReference[State](State())
/**
* Returns true if the connection is considered to be up and healthy
* and returns false otherwise.
*/
def isAvailable(connection: InetSocketAddress): Boolean = phi(connection) < threshold
/**
* Records a heartbeat for a connection.
*/
@tailrec
final def heartbeat(connection: InetSocketAddress) {
val oldState = state.get
val latestTimestamp = oldState.timestamps.get(connection)
if (latestTimestamp.isEmpty) {
// this is heartbeat from a new connection
// add starter records for this new connection
val failureStats = oldState.failureStats + (connection -> FailureStats())
val intervalHistory = oldState.intervalHistory + (connection -> Vector.empty[Long])
val timestamps = oldState.timestamps + (connection -> newTimestamp)
val newState = oldState copy (version = oldState.version + 1,
failureStats = failureStats,
intervalHistory = intervalHistory,
timestamps = timestamps)
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) heartbeat(connection) // recur
} else {
// this is a known connection
val timestamp = newTimestamp
val interval = timestamp - latestTimestamp.get
val timestamps = oldState.timestamps + (connection -> timestamp) // record new timestamp
var newIntervalsForConnection =
oldState.intervalHistory.get(connection).getOrElse(Vector.empty[Long]) :+ interval // append the new interval to history
if (newIntervalsForConnection.size > maxSampleSize) {
// reached max history, drop first interval
newIntervalsForConnection = newIntervalsForConnection drop 0
}
val failureStats =
if (newIntervalsForConnection.size > 1) {
val mean: Double = newIntervalsForConnection.sum / newIntervalsForConnection.size.toDouble
val oldFailureStats = oldState.failureStats.get(connection).getOrElse(FailureStats())
val deviationSum =
newIntervalsForConnection
.map(_.toDouble)
.foldLeft(0.0D)((x, y) x + (y - mean))
val variance: Double = deviationSum / newIntervalsForConnection.size.toDouble
val deviation: Double = math.sqrt(variance)
val newFailureStats = oldFailureStats copy (mean = mean,
deviation = deviation,
variance = variance)
oldState.failureStats + (connection -> newFailureStats)
} else {
oldState.failureStats
}
val intervalHistory = oldState.intervalHistory + (connection -> newIntervalsForConnection)
val newState = oldState copy (version = oldState.version + 1,
failureStats = failureStats,
intervalHistory = intervalHistory,
timestamps = timestamps)
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) heartbeat(connection) // recur
}
}
/**
* Calculates how likely it is that the connection has failed.
* <p/>
* If a connection does not have any records in failure detector then it is
* considered dead. This is true either if the heartbeat have not started
* yet or the connection have been explicitly removed.
*/
def phi(connection: InetSocketAddress): Double = {
val oldState = state.get
val oldTimestamp = oldState.timestamps.get(connection)
if (oldTimestamp.isEmpty) Double.MaxValue
else {
-1 * math.log10(
probability(
connection,
newTimestamp - oldTimestamp.get,
oldState))
}
}
/**
* Removes the heartbeat management for a connection.
*/
@tailrec
final def remove(connection: InetSocketAddress) {
val oldState = state.get
if (oldState.failureStats.contains(connection)) {
val failureStats = oldState.failureStats - connection
val intervalHistory = oldState.intervalHistory - connection
val timestamps = oldState.timestamps - connection
val newState = oldState copy (version = oldState.version + 1,
failureStats = failureStats,
intervalHistory = intervalHistory,
timestamps = timestamps)
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) remove(connection) // recur
}
}
private def probability(connection: InetSocketAddress, timestamp: Long, oldState: State): Double = {
val statsForConnection = oldState.failureStats.get(connection).getOrElse(FailureStats())
val exponent = -1.0 * timestamp / statsForConnection.mean
1 - (1.0 - math.pow(math.E, exponent))
}
def recordSuccess(connection: InetSocketAddress, timestamp: Long) {}
def recordFailure(connection: InetSocketAddress, timestamp: Long) {}
def notify(event: RemoteLifeCycleEvent) {}
}

View file

@ -0,0 +1,230 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
import akka.AkkaException
import akka.actor._
import akka.event.EventHandler
import akka.config.ConfigurationException
import akka.actor.UntypedChannel._
import akka.dispatch.Future
import akka.util.ReflectiveAccess
import akka.util.Duration
import java.net.InetSocketAddress
import java.lang.reflect.InvocationTargetException
import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger }
import scala.collection.immutable.Map
import scala.collection.mutable
import scala.annotation.tailrec
/**
* The failure detector uses different heuristics (depending on implementation) to try to detect and manage
* failed connections.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait FailureDetector extends NetworkEventStream.Listener {
def newTimestamp: Long = System.currentTimeMillis
/**
* Returns true if the 'connection' is considered available.
*/
def isAvailable(connection: InetSocketAddress): Boolean
/**
* Records a successful connection.
*/
def recordSuccess(connection: InetSocketAddress, timestamp: Long)
/**
* Records a failed connection.
*/
def recordFailure(connection: InetSocketAddress, timestamp: Long)
}
/**
* Misc helper and factory methods for failure detection.
*/
object FailureDetector {
def createCustomFailureDetector(implClass: String): FailureDetector = {
ReflectiveAccess.createInstance(
implClass,
Array[Class[_]](),
Array[AnyRef]()) match {
case Right(failureDetector) failureDetector
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
throw new ConfigurationException(
"Could not instantiate custom FailureDetector of [" +
implClass + "] due to: " +
cause, cause)
}
}
}
/**
* No-op failure detector. Does not do anything.
*/
class NoOpFailureDetector extends FailureDetector {
def isAvailable(connection: InetSocketAddress): Boolean = true
def recordSuccess(connection: InetSocketAddress, timestamp: Long) {}
def recordFailure(connection: InetSocketAddress, timestamp: Long) {}
def notify(event: RemoteLifeCycleEvent) {}
}
/**
* Simple failure detector that removes the failing connection permanently on first error.
*/
class RemoveConnectionOnFirstFailureFailureDetector extends FailureDetector {
protected case class State(version: Long, banned: Set[InetSocketAddress])
protected val state: AtomicReference[State] = new AtomicReference[State](newState())
protected def newState() = State(Long.MinValue, Set.empty[InetSocketAddress])
def isAvailable(connectionAddress: InetSocketAddress): Boolean = state.get.banned.contains(connectionAddress)
final def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long) {}
@tailrec
final def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long) {
val oldState = state.get
if (!oldState.banned.contains(connectionAddress)) {
val newBannedConnections = oldState.banned + connectionAddress
val newState = oldState copy (version = oldState.version + 1, banned = newBannedConnections)
if (!state.compareAndSet(oldState, newState)) recordFailure(connectionAddress, timestamp)
}
}
// NetworkEventStream.Listener callback
def notify(event: RemoteLifeCycleEvent) = event match {
case RemoteClientWriteFailed(request, cause, client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientError(cause, client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientDisconnected(client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientShutdown(client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case _ {}
}
}
/**
* Failure detector that bans the failing connection for 'timeToBan: Duration' and will try to use the connection
* again after the ban period have expired.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class BannagePeriodFailureDetector(timeToBan: Duration) extends FailureDetector with NetworkEventStream.Listener {
// FIXME considering adding a Scheduler event to notify the BannagePeriodFailureDetector unban the banned connection after the timeToBan have exprired
protected case class State(version: Long, banned: Map[InetSocketAddress, BannedConnection])
protected val state: AtomicReference[State] = new AtomicReference[State](newState())
case class BannedConnection(bannedSince: Long, address: InetSocketAddress)
val timeToBanInMillis = timeToBan.toMillis
protected def newState() = State(Long.MinValue, Map.empty[InetSocketAddress, BannedConnection])
private def bannedConnections = state.get.banned
def isAvailable(connectionAddress: InetSocketAddress): Boolean = bannedConnections.get(connectionAddress).isEmpty
@tailrec
final def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long) {
val oldState = state.get
val bannedConnection = oldState.banned.get(connectionAddress)
if (bannedConnection.isDefined) { // is it banned or not?
val BannedConnection(bannedSince, banned) = bannedConnection.get
val currentlyBannedFor = newTimestamp - bannedSince
if (currentlyBannedFor > timeToBanInMillis) {
val newBannedConnections = oldState.banned - connectionAddress
val newState = oldState copy (version = oldState.version + 1, banned = newBannedConnections)
if (!state.compareAndSet(oldState, newState)) recordSuccess(connectionAddress, timestamp)
}
}
}
@tailrec
final def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long) {
val oldState = state.get
val connection = oldState.banned.get(connectionAddress)
if (connection.isEmpty) { // is it already banned or not?
val bannedConnection = BannedConnection(timestamp, connectionAddress)
val newBannedConnections = oldState.banned + (connectionAddress -> bannedConnection)
val newState = oldState copy (version = oldState.version + 1, banned = newBannedConnections)
if (!state.compareAndSet(oldState, newState)) recordFailure(connectionAddress, timestamp)
}
}
// NetworkEventStream.Listener callback
def notify(event: RemoteLifeCycleEvent) = event match {
case RemoteClientStarted(client, connectionAddress)
recordSuccess(connectionAddress, newTimestamp)
case RemoteClientConnected(client, connectionAddress)
recordSuccess(connectionAddress, newTimestamp)
case RemoteClientWriteFailed(request, cause, client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientError(cause, client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientDisconnected(client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientShutdown(client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case _ {}
}
}
/**
* Failure detector that uses the Circuit Breaker pattern to detect and recover from failing connections.
*
* class CircuitBreakerNetworkEventStream.Listener(initialConnections: Map[InetSocketAddress, ActorRef])
* extends RemoteConnectionManager(initialConnections) {
*
* def newState() = State(Long.MinValue, initialConnections, None)
*
* def isAvailable(connectionAddress: InetSocketAddress): Boolean = connections.get(connectionAddress).isDefined
*
* def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long) {}
*
* def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long) {}
*
* // FIXME implement CircuitBreakerNetworkEventStream.Listener
* }
*/

View file

@ -6,9 +6,8 @@ package akka.remote
import akka.actor._
import akka.routing._
import DeploymentConfig._
import Actor._
import Status._
import akka.actor.Actor._
import akka.actor.Status._
import akka.event.EventHandler
import akka.util.duration._
import akka.config.ConfigurationException
@ -35,7 +34,9 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
private val actors = new ConcurrentHashMap[String, Promise[Option[ActorRef]]]
private val failureDetector = new BannagePeriodFailureDetector(remote, timeToBan = 60 seconds) // FIXME make timeToBan configurable
private val remoteDaemonConnectionManager = new RemoteConnectionManager(
remote = remote,
failureDetector = new BannagePeriodFailureDetector(60 seconds)) // FIXME make timeout configurable
def actorOf(props: Props, address: String): Option[ActorRef] = {
Address.validate(address)
@ -44,9 +45,17 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
val oldFuture = actors.putIfAbsent(address, newFuture)
if (oldFuture eq null) { // we won the race -- create the actor and resolve the future
val deploymentConfig = app.deployer.deploymentConfig
val actor = try {
app.deployer.lookupDeploymentFor(address) match {
case Some(Deploy(_, _, router, nrOfInstances, _, app.deployer.deploymentConfig.RemoteScope(remoteAddresses)))
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, deploymentConfig.RemoteScope(remoteAddresses)))
val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match {
case FailureDetectorType.NoOp new NoOpFailureDetector
case FailureDetectorType.RemoveConnectionOnFirstFailure new RemoveConnectionOnFirstFailureFailureDetector
case FailureDetectorType.BannagePeriod(timeToBan) new BannagePeriodFailureDetector(timeToBan)
case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass)
}
val thisHostname = remote.address.getHostName
val thisPort = remote.address.getPort
@ -61,8 +70,7 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
} else {
// we are on the single "reference" node uses the remote actors on the replica nodes
val routerType = DeploymentConfig.routerTypeFor(router)
val routerFactory: () Router = routerType match {
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct
if (remoteAddresses.size != 1) throw new ConfigurationException(
"Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]"
@ -81,23 +89,31 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
.format(address, remoteAddresses.mkString(", ")))
() new RoundRobinRouter
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
case RouterType.Custom sys.error("Router Custom not supported yet")
case RouterType.ScatterGather
if (remoteAddresses.size < 1) throw new ConfigurationException(
"Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]"
.format(address, remoteAddresses.mkString(", ")))
() new ScatterGatherFirstCompletedRouter
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
case RouterType.Custom(implClass) () Routing.createCustomRouter(implClass)
}
def provisionActorToNode(remoteAddress: RemoteAddress): RemoteActorRef = {
var connections = Map.empty[InetSocketAddress, ActorRef]
remoteAddresses foreach { remoteAddress: DeploymentConfig.RemoteAddress
val inetSocketAddress = new InetSocketAddress(remoteAddress.hostname, remoteAddress.port)
useActorOnNode(inetSocketAddress, address, props.creator)
RemoteActorRef(app, app.remote, inetSocketAddress, address, None)
connections += (inetSocketAddress -> RemoteActorRef(app.remote, inetSocketAddress, address, None))
}
val connections: Iterable[ActorRef] = remoteAddresses map { provisionActorToNode(_) }
val connectionManager = new RemoteConnectionManager(remote, connections, failureDetector)
Some(app.routing.actorOf(RoutedProps(
connections.keys foreach { useActorOnNode(_, address, props.creator) }
Some(app.createActor(RoutedProps(
routerFactory = routerFactory,
connections = connections)))
connectionManager = connectionManager)))
}
case deploy None // non-remote actor
@ -108,7 +124,7 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
throw e
}
actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later
// actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later
newFuture completeWithResult actor
actor
@ -118,7 +134,18 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
}
}
def findActorRef(address: String): Option[ActorRef] = throw new UnsupportedOperationException
/**
* Copied from LocalActorRefProvider...
*/
def actorOf(props: RoutedProps, address: String): Option[ActorRef] = {
if (props.connectionManager.size == 0) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router")
Some(new RoutedActorRef(props, address))
}
def actorFor(address: String): Option[ActorRef] = actors.get(address) match {
case null None
case future future.await.resultOrException.getOrElse(None)
}
/**
* Returns true if the actor was in the provider's cache and evicted successfully, else false.
@ -150,7 +177,7 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
remote.remoteDaemonServiceName, remoteAddress.getHostName, remoteAddress.getPort)
// try to get the connection for the remote address, if not already there then create it
val connection = failureDetector.putIfAbsent(remoteAddress, connectionFactory)
val connection = remoteDaemonConnectionManager.putIfAbsent(remoteAddress, connectionFactory)
sendCommandToRemoteNode(connection, command, withACK = true) // ensure we get an ACK on the USE command
}

View file

@ -0,0 +1,149 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
import akka.actor._
import akka.routing._
import akka.event.EventHandler
import scala.collection.immutable.Map
import scala.annotation.tailrec
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
/**
* Remote connection manager, manages remote connections, e.g. RemoteActorRef's.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteConnectionManager(
remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef],
failureDetector: FailureDetector = new NoOpFailureDetector)
extends ConnectionManager {
case class State(version: Long, connections: Map[InetSocketAddress, ActorRef])
extends VersionedIterable[ActorRef] {
def iterable: Iterable[ActorRef] = connections.values
}
private val state: AtomicReference[State] = new AtomicReference[State](newState())
// register all initial connections - e.g listen to events from them
initialConnections.keys foreach (remote.eventStream.register(failureDetector, _))
/**
* This method is using the FailureDetector to filter out connections that are considered not available.
*/
private def filterAvailableConnections(current: State): State = {
val availableConnections = current.connections filter { entry failureDetector.isAvailable(entry._1) }
current copy (version = current.version, connections = availableConnections)
}
private def newState() = State(Long.MinValue, initialConnections)
def version: Long = state.get.version
def connections = filterAvailableConnections(state.get)
def size: Int = connections.connections.size
def shutdown() {
state.get.iterable foreach (_.stop()) // shut down all remote connections
}
@tailrec
final def failOver(from: InetSocketAddress, to: InetSocketAddress) {
EventHandler.debug(this, "Failing over connection from [%s] to [%s]".format(from, to))
val oldState = state.get
var changed = false
val newMap = oldState.connections map {
case (`from`, actorRef)
changed = true
//actorRef.stop()
(to, newConnection(actorRef.address, to))
case other other
}
if (changed) {
//there was a state change, so we are now going to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newMap)
//if we are not able to update, the state, we are going to try again.
if (!state.compareAndSet(oldState, newState)) {
failOver(from, to) // recur
}
}
}
@tailrec
final def remove(faultyConnection: ActorRef) {
val oldState = state.get()
var changed = false
var faultyAddress: InetSocketAddress = null
var newConnections = Map.empty[InetSocketAddress, ActorRef]
oldState.connections.keys foreach { address
val actorRef: ActorRef = oldState.connections.get(address).get
if (actorRef ne faultyConnection) {
newConnections = newConnections + ((address, actorRef))
} else {
faultyAddress = address
changed = true
}
}
if (changed) {
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) {
remove(faultyConnection) // recur
} else {
EventHandler.debug(this, "Removing connection [%s]".format(faultyAddress))
remote.eventStream.unregister(failureDetector, faultyAddress) // unregister the connections - e.g stop listen to events from it
}
}
}
@tailrec
final def putIfAbsent(address: InetSocketAddress, newConnectionFactory: () ActorRef): ActorRef = {
val oldState = state.get()
val oldConnections = oldState.connections
oldConnections.get(address) match {
case Some(connection) connection // we already had the connection, return it
case None // we need to create it
val newConnection = newConnectionFactory()
val newConnections = oldConnections + (address -> newConnection)
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) {
// we failed, need compensating action
newConnection.stop() // stop the new connection actor and try again
putIfAbsent(address, newConnectionFactory) // recur
} else {
// we succeeded
EventHandler.debug(this, "Adding connection [%s]".format(address))
remote.eventStream.register(failureDetector, address) // register the connection - e.g listen to events from it
newConnection // return new connection actor
}
}
}
private[remote] def newConnection(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
RemoteActorRef(remote.server, inetSocketAddress, actorAddress, None)
}
}

View file

@ -1,380 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
import akka.actor._
import Actor._
import akka.routing._
import akka.dispatch.PinnedDispatcher
import akka.event.EventHandler
import akka.util.{ ListenerManagement, Duration }
import scala.collection.immutable.Map
import scala.collection.mutable
import scala.annotation.tailrec
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import System.{ currentTimeMillis newTimestamp }
/**
* Base class for remote failure detection management.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
abstract class RemoteFailureDetectorBase(remote: Remote, initialConnections: Map[InetSocketAddress, ActorRef])
extends FailureDetector
with NetworkEventStream.Listener {
type T <: AnyRef
protected case class State(
version: Long,
connections: Map[InetSocketAddress, ActorRef],
meta: T = null.asInstanceOf[T])
extends VersionedIterable[ActorRef] {
def iterable: Iterable[ActorRef] = connections.values
}
protected val state: AtomicReference[State] = new AtomicReference[State](newState())
// register all initial connections - e.g listen to events from them
initialConnections.keys foreach (remote.eventStream.register(this, _))
/**
* State factory. To be defined by subclass that wants to add extra info in the 'meta: T' field.
*/
protected def newState(): State
/**
* Returns true if the 'connection' is considered available.
*
* To be implemented by subclass.
*/
def isAvailable(connectionAddress: InetSocketAddress): Boolean
/**
* Records a successful connection.
*
* To be implemented by subclass.
*/
def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long)
/**
* Records a failed connection.
*
* To be implemented by subclass.
*/
def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long)
def version: Long = state.get.version
def versionedIterable = state.get
def size: Int = state.get.connections.size
def connections: Map[InetSocketAddress, ActorRef] = state.get.connections
def stopAll() {
state.get.iterable foreach (_.stop()) // shut down all remote connections
}
@tailrec
final def failOver(from: InetSocketAddress, to: InetSocketAddress) {
EventHandler.debug(this, "RemoteFailureDetector failover from [%s] to [%s]".format(from, to))
val oldState = state.get
var changed = false
val newMap = oldState.connections map {
case (`from`, actorRef)
changed = true
//actorRef.stop()
(to, newConnection(actorRef.address, to))
case other other
}
if (changed) {
//there was a state change, so we are now going to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newMap)
//if we are not able to update, the state, we are going to try again.
if (!state.compareAndSet(oldState, newState)) {
failOver(from, to) // recur
}
}
}
@tailrec
final def remove(faultyConnection: ActorRef) {
val oldState = state.get()
var changed = false
var faultyAddress: InetSocketAddress = null
var newConnections = Map.empty[InetSocketAddress, ActorRef]
oldState.connections.keys foreach { address
val actorRef: ActorRef = oldState.connections.get(address).get
if (actorRef ne faultyConnection) {
newConnections = newConnections + ((address, actorRef))
} else {
faultyAddress = address
changed = true
}
}
if (changed) {
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) {
remove(faultyConnection) // recur
} else {
EventHandler.debug(this, "Removing connection [%s]".format(faultyAddress))
remote.eventStream.unregister(this, faultyAddress) // unregister the connections - e.g stop listen to events from it
}
}
}
@tailrec
final def putIfAbsent(address: InetSocketAddress, newConnectionFactory: () ActorRef): ActorRef = {
val oldState = state.get()
val oldConnections = oldState.connections
oldConnections.get(address) match {
case Some(connection) connection // we already had the connection, return it
case None // we need to create it
val newConnection = newConnectionFactory()
val newConnections = oldConnections + (address -> newConnection)
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) {
// we failed, need compensating action
newConnection.stop() // stop the new connection actor and try again
putIfAbsent(address, newConnectionFactory) // recur
} else {
// we succeeded
EventHandler.debug(this, "Adding connection [%s]".format(address))
remote.eventStream.register(this, address) // register the connection - e.g listen to events from it
newConnection // return new connection actor
}
}
}
private[remote] def newConnection(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
RemoteActorRef(remote.app, remote.server, inetSocketAddress, actorAddress, None)
}
}
/**
* Simple failure detector that removes the failing connection permanently on first error.
*/
class RemoveConnectionOnFirstFailureRemoteFailureDetector(_remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef])
extends RemoteFailureDetectorBase(_remote, initialConnections) {
protected def newState() = State(Long.MinValue, initialConnections)
def isAvailable(connectionAddress: InetSocketAddress): Boolean = connections.get(connectionAddress).isDefined
def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long) {}
def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long) {}
def notify(event: RemoteLifeCycleEvent) = event match {
case RemoteClientWriteFailed(request, cause, client, connectionAddress)
removeConnection(connectionAddress)
case RemoteClientError(cause, client, connectionAddress)
removeConnection(connectionAddress)
case RemoteClientDisconnected(client, connectionAddress)
removeConnection(connectionAddress)
case RemoteClientShutdown(client, connectionAddress)
removeConnection(connectionAddress)
case _ {}
}
private def removeConnection(connectionAddress: InetSocketAddress) =
connections.get(connectionAddress) foreach { conn remove(conn) }
}
/**
* Failure detector that bans the failing connection for 'timeToBan: Duration' and will try to use the connection
* again after the ban period have expired.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class BannagePeriodFailureDetector(_remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef],
timeToBan: Duration)
extends RemoteFailureDetectorBase(_remote, initialConnections) {
// FIXME considering adding a Scheduler event to notify the BannagePeriodFailureDetector unban the banned connection after the timeToBan have exprired
type T = Map[InetSocketAddress, BannedConnection]
case class BannedConnection(bannedSince: Long, connection: ActorRef)
val timeToBanInMillis = timeToBan.toMillis
protected def newState() =
State(Long.MinValue, initialConnections, Map.empty[InetSocketAddress, BannedConnection])
private def removeConnection(connectionAddress: InetSocketAddress) =
connections.get(connectionAddress) foreach { conn remove(conn) }
// ===================================================================================
// FailureDetector callbacks
// ===================================================================================
def isAvailable(connectionAddress: InetSocketAddress): Boolean = connections.get(connectionAddress).isDefined
@tailrec
final def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long) {
val oldState = state.get
val bannedConnection = oldState.meta.get(connectionAddress)
if (bannedConnection.isDefined) {
val BannedConnection(bannedSince, connection) = bannedConnection.get
val currentlyBannedFor = newTimestamp - bannedSince
if (currentlyBannedFor > timeToBanInMillis) {
// ban time has expired - add connection to available connections
val newConnections = oldState.connections + (connectionAddress -> connection)
val newBannedConnections = oldState.meta - connectionAddress
val newState = oldState copy (version = oldState.version + 1,
connections = newConnections,
meta = newBannedConnections)
if (!state.compareAndSet(oldState, newState)) recordSuccess(connectionAddress, timestamp)
}
}
}
@tailrec
final def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long) {
val oldState = state.get
val connection = oldState.connections.get(connectionAddress)
if (connection.isDefined) {
val newConnections = oldState.connections - connectionAddress
val bannedConnection = BannedConnection(timestamp, connection.get)
val newBannedConnections = oldState.meta + (connectionAddress -> bannedConnection)
val newState = oldState copy (version = oldState.version + 1,
connections = newConnections,
meta = newBannedConnections)
if (!state.compareAndSet(oldState, newState)) recordFailure(connectionAddress, timestamp)
}
}
// ===================================================================================
// NetworkEventStream.Listener callback
// ===================================================================================
def notify(event: RemoteLifeCycleEvent) = event match {
case RemoteClientStarted(client, connectionAddress)
recordSuccess(connectionAddress, newTimestamp)
case RemoteClientConnected(client, connectionAddress)
recordSuccess(connectionAddress, newTimestamp)
case RemoteClientWriteFailed(request, cause, client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientError(cause, client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientDisconnected(client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case RemoteClientShutdown(client, connectionAddress)
recordFailure(connectionAddress, newTimestamp)
case _ {}
}
}
/**
* Failure detector that uses the Circuit Breaker pattern to detect and recover from failing connections.
*
* class CircuitBreakerNetworkEventStream.Listener(initialConnections: Map[InetSocketAddress, ActorRef])
* extends RemoteFailureDetectorBase(initialConnections) {
*
* def newState() = State(Long.MinValue, initialConnections, None)
*
* def isAvailable(connectionAddress: InetSocketAddress): Boolean = connections.get(connectionAddress).isDefined
*
* def recordSuccess(connectionAddress: InetSocketAddress, timestamp: Long) {}
*
* def recordFailure(connectionAddress: InetSocketAddress, timestamp: Long) {}
*
* // FIXME implement CircuitBreakerNetworkEventStream.Listener
* }
*/
/**
* Base trait for remote failure event listener.
*/
trait RemoteFailureListener {
final private[akka] def notify(event: RemoteLifeCycleEvent) = event match {
case RemoteClientStarted(client, connectionAddress)
remoteClientStarted(client, connectionAddress)
case RemoteClientConnected(client, connectionAddress)
remoteClientConnected(client, connectionAddress)
case RemoteClientWriteFailed(request, cause, client, connectionAddress)
remoteClientWriteFailed(request, cause, client, connectionAddress)
case RemoteClientError(cause, client, connectionAddress)
remoteClientError(cause, client, connectionAddress)
case RemoteClientDisconnected(client, connectionAddress)
remoteClientDisconnected(client, connectionAddress)
case RemoteClientShutdown(client, connectionAddress)
remoteClientShutdown(client, connectionAddress)
case RemoteServerWriteFailed(request, cause, server, clientAddress)
remoteServerWriteFailed(request, cause, server, clientAddress)
case RemoteServerError(cause, server)
remoteServerError(cause, server)
case RemoteServerShutdown(server)
remoteServerShutdown(server)
}
def remoteClientStarted(client: RemoteClientModule, connectionAddress: InetSocketAddress) {}
def remoteClientConnected(client: RemoteClientModule, connectionAddress: InetSocketAddress) {}
def remoteClientWriteFailed(
request: AnyRef, cause: Throwable, client: RemoteClientModule, connectionAddress: InetSocketAddress) {}
def remoteClientError(cause: Throwable, client: RemoteClientModule, connectionAddress: InetSocketAddress) {}
def remoteClientDisconnected(client: RemoteClientModule, connectionAddress: InetSocketAddress) {}
def remoteClientShutdown(client: RemoteClientModule, connectionAddress: InetSocketAddress) {}
def remoteServerWriteFailed(
request: AnyRef, cause: Throwable, server: RemoteServerModule, clientAddress: Option[InetSocketAddress]) {}
def remoteServerError(cause: Throwable, server: RemoteServerModule) {}
def remoteServerShutdown(server: RemoteServerModule) {}
}

View file

@ -52,7 +52,6 @@ trait NettyRemoteClientModule extends RemoteClientModule {
self: RemoteSupport
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val remoteActors = new Index[RemoteAddress, Uuid]
private val lock = new ReadWriteGuard
def app: AkkaApplication
@ -144,22 +143,13 @@ abstract class RemoteClient private[akka] (
val module: NettyRemoteClientModule,
val remoteAddress: InetSocketAddress) {
import app.config
implicit def _app = app
val serialization = new RemoteActorSerialization(app)
val useTransactionLog = config.getBool("akka.remote.client.buffering.retry-message-send-on-failure", false)
val transactionLogCapacity = config.getInt("akka.remote.client.buffering.capacity", -1)
val name = this.getClass.getSimpleName + "@" +
remoteAddress.getAddress.getHostAddress + "::" +
remoteAddress.getPort
val serialization = new RemoteActorSerialization(app)
protected val futures = new ConcurrentHashMap[Uuid, Promise[_]]
protected val pendingRequests = {
if (transactionLogCapacity < 0) new ConcurrentLinkedQueue[(Boolean, Uuid, RemoteMessageProtocol)]
new LinkedBlockingQueue[(Boolean, Uuid, RemoteMessageProtocol)](transactionLogCapacity)
}
private[remote] val runSwitch = new Switch()
@ -173,19 +163,6 @@ abstract class RemoteClient private[akka] (
def shutdown(): Boolean
/**
* Returns an array with the current pending messages not yet delivered.
*/
def pendingMessages: Array[Any] = {
var messages = Vector[Any]()
val iter = pendingRequests.iterator
while (iter.hasNext) {
val (_, _, message) = iter.next
messages = messages :+ MessageSerializer.deserialize(app, message.getMessage)
}
messages.toArray
}
/**
* Converts the message to the wireprotocol and sends the message across the wire
*/
@ -220,13 +197,7 @@ abstract class RemoteClient private[akka] (
notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress))
}
} catch {
case e: Exception
notifyListeners(RemoteClientError(e, module, remoteAddress))
if (useTransactionLog && !pendingRequests.offer((true, null, request))) { // Add the request to the tx log after a failing send
pendingRequests.clear()
throw new RemoteClientMessageBufferException("Buffer limit [" + transactionLogCapacity + "] reached")
}
case e: Exception notifyListeners(RemoteClientError(e, module, remoteAddress))
}
None
@ -234,20 +205,14 @@ abstract class RemoteClient private[akka] (
} else {
val futureResult =
if (senderFuture.isDefined) senderFuture.get
else new DefaultPromise[T](request.getActorInfo.getTimeout)
else new DefaultPromise[T](request.getActorInfo.getTimeout)(app.dispatcher)
val futureUuid = uuidFrom(request.getUuid.getHigh, request.getUuid.getLow)
futures.put(futureUuid, futureResult) // Add future prematurely, remove it if write fails
def handleRequestReplyError(future: ChannelFuture) = {
if (useTransactionLog && !pendingRequests.offer((false, futureUuid, request))) { // Add the request to the tx log after a failing send
pendingRequests.clear()
throw new RemoteClientMessageBufferException("Buffer limit [" + transactionLogCapacity + "] reached")
} else {
val f = futures.remove(futureUuid) // Clean up future
if (f ne null) f.completeWithException(future.getCause)
}
val f = futures.remove(futureUuid) // Clean up future
if (f ne null) f.completeWithException(future.getCause)
}
var future: ChannelFuture = null
@ -275,41 +240,6 @@ abstract class RemoteClient private[akka] (
throw exception
}
}
private[remote] def sendPendingRequests() = pendingRequests synchronized {
// ensure only one thread at a time can flush the log
val nrOfMessages = pendingRequests.size
if (nrOfMessages > 0) EventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages)
var pendingRequest = pendingRequests.peek
while (pendingRequest ne null) {
val (isOneWay, futureUuid, message) = pendingRequest
if (isOneWay) {
// tell
val future = currentChannel.write(RemoteEncoder.encode(message))
future.awaitUninterruptibly()
if (future.isCancelled && !future.isSuccess) {
notifyListeners(RemoteClientWriteFailed(message, future.getCause, module, remoteAddress))
}
} else {
// ask
val future = currentChannel.write(RemoteEncoder.encode(message))
future.awaitUninterruptibly()
if (future.isCancelled || !future.isSuccess) {
val f = futures.remove(futureUuid) // Clean up future
if (f ne null) f.completeWithException(future.getCause)
notifyListeners(RemoteClientWriteFailed(message, future.getCause, module, remoteAddress))
}
}
pendingRequests.remove(pendingRequest)
pendingRequest = pendingRequests.peek // try to grab next message
}
}
}
/**
@ -442,7 +372,6 @@ class ActiveRemoteClient private[akka] (
bootstrap.releaseExternalResources()
bootstrap = null
connection = null
pendingRequests.clear()
EventHandler.info(this, "[%s] has been shut down".format(name))
}
@ -565,7 +494,6 @@ class ActiveRemoteClientHandler(
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
try {
if (client.useTransactionLog) client.sendPendingRequests() // try to send pending requests (still there after client/server crash ard reconnect
client.notifyListeners(RemoteClientConnected(client.module, client.remoteAddress))
EventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress))
client.resetReconnectionTimeWindow
@ -649,7 +577,7 @@ class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with
EventHandler.debug(this,
"Creating RemoteActorRef with address [%s] connected to [%s]"
.format(actorAddress, remoteInetSocketAddress))
RemoteActorRef(app, app.remote, remoteInetSocketAddress, actorAddress, loader)
RemoteActorRef(app.remote, remoteInetSocketAddress, actorAddress, loader)
}
}
@ -955,7 +883,7 @@ class RemoteServerHandler(
try {
actor ! PoisonPill
} catch {
case e: Exception EventHandler.error(e, this, "Couldn't stop %s".format(actor))
case e: Exception EventHandler.error(e, this, "Couldn't stop [%s]".format(actor))
}
}
@ -972,7 +900,7 @@ class RemoteServerHandler(
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = {
event.getMessage match {
case null
throw new IllegalActorStateException("Message in remote MessageEvent is null: " + event)
throw new IllegalActorStateException("Message in remote MessageEvent is null [" + event + "]")
case remote: AkkaRemoteProtocol if remote.hasMessage
handleRemoteMessageProtocol(remote.getMessage, event.getChannel)
@ -1071,12 +999,6 @@ class RemoteServerHandler(
private def createActor(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = {
val uuid = actorInfo.getUuid
val address = actorInfo.getAddress
// val address = {
// // strip off clusterActorRefPrefix if needed
// val addr = actorInfo.getAddress
// if (addr.startsWith(Address.clusterActorRefPrefix)) addr.substring(addr.indexOf('.') + 1, addr.length)
// else addr
// }
EventHandler.debug(this,
"Looking up a remotely available actor for address [%s] on node [%s]"

View file

@ -243,7 +243,7 @@ class RemoteActorSerialization(val app: AkkaApplication) {
EventHandler.debug(this, "Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s".format(protocol))
val ref = RemoteActorRef(
app, app.remote,
app.remote,
JavaSerializer.fromBinary(protocol.getInetSocketAddress.toByteArray, Some(classOf[InetSocketAddress]), loader).asInstanceOf[InetSocketAddress],
protocol.getAddress,
loader)

View file

@ -0,0 +1,92 @@
package akka.remote
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import java.net.InetSocketAddress
class AccrualFailureDetectorSpec extends WordSpec with MustMatchers {
"An AccrualFailureDetector" should {
"mark node as available after a series of successful heartbeats" in {
val fd = new AccrualFailureDetector
val conn = new InetSocketAddress("localhost", 2552)
fd.heartbeat(conn)
Thread.sleep(1000)
fd.heartbeat(conn)
Thread.sleep(100)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
}
"mark node as dead after explicit removal of connection" in {
val fd = new AccrualFailureDetector
val conn = new InetSocketAddress("localhost", 2552)
fd.heartbeat(conn)
Thread.sleep(1000)
fd.heartbeat(conn)
Thread.sleep(100)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
fd.remove(conn)
fd.isAvailable(conn) must be(false)
}
"mark node as dead if heartbeat are missed" in {
val fd = new AccrualFailureDetector(threshold = 3)
val conn = new InetSocketAddress("localhost", 2552)
fd.heartbeat(conn)
Thread.sleep(1000)
fd.heartbeat(conn)
Thread.sleep(100)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
Thread.sleep(5000)
fd.isAvailable(conn) must be(false)
}
"mark node as available if it starts heartbeat again after being marked dead due to detection of failure" in {
val fd = new AccrualFailureDetector(threshold = 3)
val conn = new InetSocketAddress("localhost", 2552)
fd.heartbeat(conn)
Thread.sleep(1000)
fd.heartbeat(conn)
Thread.sleep(100)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
Thread.sleep(5000)
fd.isAvailable(conn) must be(false)
fd.heartbeat(conn)
Thread.sleep(1000)
fd.heartbeat(conn)
Thread.sleep(100)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
}
}
}

View file

@ -7,7 +7,7 @@ package akka.remote
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.remote.netty.NettyRemoteSupport
import akka.actor.{ Actor, ActorRegistry }
import akka.actor.Actor
import akka.testkit.AkkaSpec
import akka.dispatch.Future

View file

@ -85,5 +85,4 @@ object TestFSMRef {
def apply[S, D, T <: Actor](factory: T, address: String)(implicit ev: T <:< FSM[S, D], application: AkkaApplication): TestFSMRef[S, D, T] =
new TestFSMRef(application, Props(creator = () factory), address)
}

View file

@ -25,7 +25,7 @@ class TestFSMRefSpec extends AkkaSpec {
when(2) {
case Ev("back") goto(1) using "back"
}
})
}, "test-fsm-ref-1")
fsm.stateName must be(1)
fsm.stateData must be("")
fsm ! "go"
@ -49,14 +49,12 @@ class TestFSMRefSpec extends AkkaSpec {
when(1) {
case x stay
}
})
}, "test-fsm-ref-2")
fsm.timerActive_?("test") must be(false)
fsm.setTimer("test", 12, 10 millis, true)
fsm.timerActive_?("test") must be(true)
fsm.cancelTimer("test")
fsm.timerActive_?("test") must be(false)
}
}
}

View file

@ -8,10 +8,12 @@ import static akka.actor.Actors.poisonPill;
import static java.util.Arrays.asList;
import akka.actor.ActorRef;
import akka.actor.Actors;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.routing.RoutedProps;
import akka.routing.RouterType;
import akka.routing.LocalConnectionManager;
import akka.routing.Routing;
import akka.routing.Routing.Broadcast;
import scala.collection.JavaConversions;
@ -22,7 +24,7 @@ import java.util.concurrent.CountDownLatch;
import akka.AkkaApplication;
public class Pi {
private static final AkkaApplication app = new AkkaApplication();
public static void main(String[] args) throws Exception {
@ -112,7 +114,7 @@ public class Pi {
workers.add(worker);
}
router = app.routing().actorOf(RoutedProps.apply().withRoundRobinRouter().withConnections(workers), "pi");
router = app.createActor(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi");
}
// message handler

View file

@ -61,7 +61,7 @@ object Pi extends App {
val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker])
// wrap them with a load-balancing router
val router = app.routing.actorOf(RoutedProps().withRoundRobinRouter.withConnections(workers), "pi")
val router = app.createActor(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi")
// message handler
def receive = {

View file

@ -11,8 +11,10 @@ import static java.util.Arrays.asList;
import akka.AkkaApplication;
import akka.routing.RoutedProps;
import akka.routing.Routing;
import akka.routing.LocalConnectionManager;
import scala.Option;
import akka.actor.ActorRef;
import akka.actor.Actors;
import akka.actor.Channel;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
@ -24,7 +26,7 @@ import scala.collection.JavaConversions;
import java.util.LinkedList;
public class Pi {
private static final AkkaApplication app = new AkkaApplication();
public static void main(String[] args) throws Exception {
@ -105,7 +107,7 @@ public class Pi {
workers.add(worker);
}
router = app.routing().actorOf(RoutedProps.apply().withConnections(workers).withRoundRobinRouter(), "pi");
router = app.createActor(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi");
}
@Override

View file

@ -9,7 +9,7 @@ import akka.event.EventHandler
import System.{ currentTimeMillis now }
import akka.routing.Routing.Broadcast
import akka.actor.{ Timeout, Channel, Actor, PoisonPill }
import akka.routing.{ RoutedProps, Routing }
import akka.routing._
import akka.AkkaApplication
object Pi extends App {
@ -56,7 +56,9 @@ object Pi extends App {
val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker])
// wrap them with a load-balancing router
val router = app.routing.actorOf(RoutedProps().withConnections(workers).withRoundRobinRouter, "pi")
val router = app.createActor(RoutedProps(
routerFactory = () new RoundRobinRouter,
connectionManager = new LocalConnectionManager(workers)), "pi")
// phase 1, can accept a Calculate message
def scatter: Receive = {

View file

@ -63,8 +63,8 @@ akka {
service-ping { # deployment id pattern
router = "least-cpu" # routing (load-balance) scheme to use
# available: "direct", "round-robin", "random",
router = "round-robin" # routing (load-balance) scheme to use
# available: "direct", "round-robin", "random", "scatter-gather"
# "least-cpu", "least-ram", "least-messages"
# or: fully qualified class name of the router class
# default is "direct";
@ -76,7 +76,7 @@ akka {
# if the "direct" router is used then this element is ignored (always '1')
failure-detector { # failure detection scheme to use
bannage-period { # available: remove-connection-on-first-local-failure {}
bannage-period { # available: no-op {}
time-to-ban = 10 # remove-connection-on-first-failure {}
} # bannage-period { ... }