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

This commit is contained in:
phaller 2012-06-25 14:27:36 +02:00
commit 3337dc1387
191 changed files with 8456 additions and 3198 deletions

View file

@ -0,0 +1,17 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor;
public class NonPublicClass {
public static Props createProps() {
return new Props(MyNonPublicActorClass.class);
}
}
class MyNonPublicActorClass extends UntypedActor {
@Override public void onReceive(Object msg) {
getSender().tell(msg);
}
}

View file

@ -227,7 +227,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
contextStackMustBeEmpty
}
filterException[java.lang.IllegalStateException] {
EventFilter[ActorInitializationException](occurrences = 1) intercept {
(intercept[java.lang.IllegalStateException] {
wrap(result
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result)))))))
@ -257,14 +257,14 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
val in = new ObjectInputStream(new ByteArrayInputStream(bytes))
val readA = in.readObject
a.isInstanceOf[LocalActorRef] must be === true
readA.isInstanceOf[LocalActorRef] must be === true
a.isInstanceOf[ActorRefWithCell] must be === true
readA.isInstanceOf[ActorRefWithCell] must be === true
(readA eq a) must be === true
}
val ser = new JavaSerializer(esys)
val readA = ser.fromBinary(bytes, None)
readA.isInstanceOf[LocalActorRef] must be === true
readA.isInstanceOf[ActorRefWithCell] must be === true
(readA eq a) must be === true
}
@ -358,17 +358,24 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
system.stop(serverRef)
}
"support actorOfs where the class of the actor isn't public" in {
val a = system.actorOf(NonPublicClass.createProps())
a.tell("pigdog", testActor)
expectMsg("pigdog")
system stop a
}
"stop when sent a poison pill" in {
val timeout = Timeout(20000)
val ref = system.actorOf(Props(new Actor {
def receive = {
case 5 sender.tell("five")
case null sender.tell("null")
case 0 sender.tell("null")
}
}))
val ffive = (ref.ask(5)(timeout)).mapTo[String]
val fnull = (ref.ask(null)(timeout)).mapTo[String]
val fnull = (ref.ask(0)(timeout)).mapTo[String]
ref ! PoisonPill
Await.result(ffive, timeout.duration) must be("five")

View file

@ -10,6 +10,9 @@ import akka.dispatch.Await
import akka.util.duration._
import scala.collection.JavaConverters
import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue }
import akka.pattern.ask
import akka.util.Timeout
import akka.dispatch.Future
class JavaExtensionSpec extends JavaExtension with JUnitSuite
@ -21,8 +24,46 @@ object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider
// Dont't place inside ActorSystemSpec object, since it will not be garbage collected and reference to system remains
class TestExtension(val system: ExtendedActorSystem) extends Extension
object ActorSystemSpec {
class Waves extends Actor {
var master: ActorRef = _
var terminaters = Set[ActorRef]()
def receive = {
case n: Int
master = sender
terminaters = Set() ++ (for (i 1 to n) yield {
val man = context.watch(context.system.actorOf(Props[Terminater]))
man ! "run"
man
})
case Terminated(child) if terminaters contains child
terminaters -= child
if (terminaters.isEmpty) {
master ! "done"
context stop self
}
}
override def preRestart(cause: Throwable, msg: Option[Any]) {
if (master ne null) {
master ! "failed with " + cause + " while processing " + msg
}
context stop self
}
}
class Terminater extends Actor {
def receive = {
case "run" context.stop(self)
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") {
class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") with ImplicitSender {
"An ActorSystem" must {
@ -112,6 +153,35 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
}.getMessage must be("Must be called prior to system shutdown.")
}
"reliably create waves of actors" in {
import system.dispatcher
implicit val timeout = Timeout(30 seconds)
val waves = for (i 1 to 3) yield system.actorOf(Props[ActorSystemSpec.Waves]) ? 50000
Await.result(Future.sequence(waves), timeout.duration + 5.seconds) must be === Seq("done", "done", "done")
}
"reliable deny creation of actors while shutting down" in {
val system = ActorSystem()
system.scheduler.scheduleOnce(200 millis) { system.shutdown() }
var failing = false
var created = Vector.empty[ActorRef]
while (!system.isTerminated && system.uptime < 5) {
try {
val t = system.actorOf(Props[ActorSystemSpec.Terminater])
failing must not be true // because once failing => always failing (its due to shutdown)
created :+= t
} catch {
case _: IllegalStateException failing = true
}
}
if (system.uptime >= 5) {
println(created.last)
println(system.asInstanceOf[ExtendedActorSystem].printTree)
system.uptime must be < 5L
}
created filter (ref !ref.isTerminated && !ref.asInstanceOf[ActorRefWithCell].underlying.isInstanceOf[UnstartedCell]) must be(Seq())
}
}
}

View file

@ -6,7 +6,7 @@ package akka.actor
import akka.testkit._
import akka.testkit.DefaultTimeout
import akka.testkit.TestEvent._
import akka.dispatch.{ Await, MessageQueueAppendFailedException, BoundedDequeBasedMailbox }
import akka.dispatch.{ Await, BoundedDequeBasedMailbox }
import akka.pattern.ask
import akka.util.duration._
import akka.actor.ActorSystem.Settings
@ -17,16 +17,8 @@ object ActorWithBoundedStashSpec {
class StashingActor(implicit sys: ActorSystem) extends Actor with Stash {
def receive = {
case "hello"
stash()
sender ! "OK"
case "world"
try {
unstashAll()
} catch {
case e: MessageQueueAppendFailedException
expectedException.open()
}
case "hello" stash()
case "world" unstashAll()
}
}
@ -36,17 +28,9 @@ object ActorWithBoundedStashSpec {
def receive = {
case "hello"
numStashed += 1
try {
stash()
} catch {
case e: StashOverflowException
if (numStashed == 21) stashOverflow.open()
try stash() catch { case e: StashOverflowException if (numStashed == 21) sender ! "STASHOVERFLOW" }
}
}
}
@volatile var expectedException: TestLatch = null
@volatile var stashOverflow: TestLatch = null
val testConf: Config = ConfigFactory.parseString("""
my-dispatcher {
@ -56,47 +40,42 @@ object ActorWithBoundedStashSpec {
""")
// bounded deque-based mailbox with capacity 10
class Bounded(settings: Settings, config: Config) extends BoundedDequeBasedMailbox(10, 5 seconds)
class Bounded(settings: Settings, config: Config) extends BoundedDequeBasedMailbox(10, 1 seconds)
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorWithBoundedStashSpec extends AkkaSpec(ActorWithBoundedStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach {
class ActorWithBoundedStashSpec extends AkkaSpec(ActorWithBoundedStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach with ImplicitSender {
import ActorWithBoundedStashSpec._
implicit val sys = system
override def atStartup {
system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
}
override def atStartup { system.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) }
def myProps(creator: Actor): Props = Props(creator).withDispatcher("my-dispatcher")
"An Actor with Stash and BoundedDequeBasedMailbox" must {
"throw a MessageQueueAppendFailedException in case of a capacity violation" in {
ActorWithBoundedStashSpec.expectedException = new TestLatch
"end up in DeadLetters in case of a capacity violation" in {
system.eventStream.subscribe(testActor, classOf[DeadLetter])
val stasher = system.actorOf(myProps(new StashingActor))
// fill up stash
val futures = for (_ 1 to 11) yield { stasher ? "hello" }
futures foreach { Await.ready(_, 10 seconds) }
(1 to 11) foreach { _ stasher ! "hello" }
// cause unstashAll with capacity violation
stasher ! "world"
Await.ready(ActorWithBoundedStashSpec.expectedException, 10 seconds)
expectMsg(DeadLetter("hello", testActor, stasher))
system.eventStream.unsubscribe(testActor, classOf[DeadLetter])
}
}
"An Actor with bounded Stash" must {
"throw a StashOverflowException in case of a stash capacity violation" in {
ActorWithBoundedStashSpec.stashOverflow = new TestLatch
val stasher = system.actorOf(myProps(new StashingActorWithOverflow))
// fill up stash
for (_ 1 to 21) { stasher ! "hello" }
Await.ready(ActorWithBoundedStashSpec.stashOverflow, 10 seconds)
(1 to 21) foreach { _ stasher ! "hello" }
expectMsg("STASHOVERFLOW")
}
}
}

View file

@ -36,7 +36,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
"notify with one Terminated message when an Actor is stopped" in {
val terminal = system.actorOf(Props.empty)
startWatching(terminal) ! "hallo"
expectMsg("hallo") // this ensures that the DaemonMsgWatch has been received before we send the PoisonPill
expectMsg("hallo")
terminal ! PoisonPill
@ -143,6 +143,26 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
result must be(Seq(1, 2, 3))
}
}
"be able to watch a child with the same name after the old died" in {
val parent = system.actorOf(Props(new Actor {
def receive = {
case "NKOTB"
val currentKid = context.watch(context.actorOf(Props(ctx { case "NKOTB" ctx stop ctx.self }), "kid"))
currentKid forward "NKOTB"
context become {
case Terminated(`currentKid`)
testActor ! "GREEN"
context unbecome
}
}
}))
parent ! "NKOTB"
expectMsg("GREEN")
parent ! "NKOTB"
expectMsg("GREEN")
}
}
}

View file

@ -140,12 +140,12 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
object FSMTimingSpec {
def suspend(actorRef: ActorRef): Unit = actorRef match {
case l: LocalActorRef l.suspend()
case l: ActorRefWithCell l.suspend()
case _
}
def resume(actorRef: ActorRef): Unit = actorRef match {
case l: LocalActorRef l.resume()
case l: ActorRefWithCell l.resume()
case _
}

View file

@ -339,9 +339,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 10 seconds)(classOf[Exception] :: Nil))))
val dyingProps = Props(new Actor {
inits.incrementAndGet
if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!")
if (inits.incrementAndGet % 2 == 0) throw new IllegalStateException("Don't wanna!")
def receive = {
case Ping sender ! PongMessage
@ -366,5 +364,39 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
system.stop(supervisor)
}
"must not lose system messages when a NonFatal exception occurs when processing a system message" in {
val parent = system.actorOf(Props(new Actor {
override val supervisorStrategy = OneForOneStrategy()({
case e: IllegalStateException if e.getMessage == "OHNOES" throw e
case _ SupervisorStrategy.Restart
})
val child = context.watch(context.actorOf(Props(new Actor {
override def postRestart(reason: Throwable): Unit = testActor ! "child restarted"
def receive = {
case "die" throw new IllegalStateException("OHNOES")
case "test" sender ! "child green"
}
}), "child"))
override def postRestart(reason: Throwable): Unit = testActor ! "parent restarted"
def receive = {
case t @ Terminated(`child`) testActor ! "child terminated"
case "die" child ! "die"
case "test" sender ! "green"
case "testchild" child forward "test"
}
}))
parent ! "die"
parent ! "testchild"
expectMsg("parent restarted")
expectMsg("child terminated")
parent ! "test"
expectMsg("green")
parent ! "testchild"
expectMsg("child green")
}
}
}

View file

@ -307,7 +307,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
"be able to call methods returning Scala Options" in {
val t = newFooBar(Duration(500, "ms"))
t.optionPigdog(200).get must be("Pigdog")
t.optionPigdog(700) must be(None)
t.optionPigdog(1000) must be(None)
mustStop(t)
}

View file

@ -3,24 +3,23 @@
*/
package akka.actor.dispatch
import org.scalatest.Assertions._
import akka.testkit._
import akka.dispatch._
import akka.util.Timeout
import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit }
import akka.util.Switch
import java.rmi.RemoteException
import org.junit.{ After, Test }
import akka.actor._
import util.control.NoStackTrace
import akka.actor.ActorSystem
import akka.util.duration._
import akka.event.Logging.Error
import java.util.concurrent.{ TimeUnit, CountDownLatch, ConcurrentHashMap }
import java.util.concurrent.atomic.{ AtomicLong, AtomicInteger }
import org.junit.runner.RunWith
import org.scalatest.Assertions.{ fail, assert }
import org.scalatest.junit.JUnitRunner
import com.typesafe.config.Config
import akka.util.Duration
import akka.actor._
import akka.dispatch._
import akka.event.Logging.Error
import akka.pattern.ask
import akka.testkit._
import akka.util.{ Timeout, Switch, Duration }
import akka.util.duration._
object ActorModelSpec {
@ -201,7 +200,7 @@ object ActorModelSpec {
msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(),
restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].dispatcher))
val deadline = System.currentTimeMillis + 1000
try {
await(deadline)(stats.suspensions.get() == suspensions)
@ -241,6 +240,13 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
def newTestActor(dispatcher: String) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
def awaitStarted(ref: ActorRef): Unit = {
awaitCond(ref match {
case r: RepointableRef r.isStarted
case _ true
}, 1 second, 10 millis)
}
protected def interceptedDispatcher(): MessageDispatcherInterceptor
protected def dispatcherType: String
@ -280,6 +286,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
implicit val dispatcher = interceptedDispatcher()
val start, oneAtATime = new CountDownLatch(1)
val a = newTestActor(dispatcher.id)
awaitStarted(a)
a ! CountDown(start)
assertCountDown(start, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds")
@ -328,7 +335,8 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
"not process messages for a suspended actor" in {
implicit val dispatcher = interceptedDispatcher()
val a = newTestActor(dispatcher.id).asInstanceOf[LocalActorRef]
val a = newTestActor(dispatcher.id).asInstanceOf[InternalActorRef]
awaitStarted(a)
val done = new CountDownLatch(1)
a.suspend
a ! CountDown(done)
@ -374,7 +382,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ll.self.path compareTo rr.self.path }
} foreach {
case cell: ActorCell
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain()))
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain(null)))
}
System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages)
@ -436,6 +444,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
"not double-deregister" in {
implicit val dispatcher = interceptedDispatcher()
for (i 1 to 1000) system.actorOf(Props.empty)
val a = newTestActor(dispatcher.id)
a ! DoubleStop
awaitCond(statsFor(a, dispatcher).registers.get == 1)

View file

@ -1,8 +1,12 @@
package akka.actor.dispatch
import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.dispatch.{ Mailbox, Dispatchers }
import akka.actor.{ LocalActorRef, IllegalActorStateException, Actor, Props }
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import akka.actor.{ Props, ActorRefWithCell, ActorCell, Actor }
import akka.dispatch.Mailbox
import akka.testkit.AkkaSpec
object BalancingDispatcherSpec {
@ -51,8 +55,8 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) {
"have fast actor stealing work from slow actor" in {
val finishedCounter = new CountDownLatch(110)
val slow = system.actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef]
val fast = system.actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef]
val slow = system.actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[ActorRefWithCell]
val fast = system.actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[ActorRefWithCell]
var sentToFast = 0
@ -76,11 +80,11 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) {
}
finishedCounter.await(5, TimeUnit.SECONDS)
fast.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
slow.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be >
(slow.underlying.actor.asInstanceOf[DelayableActor].invocationCount)
fast.underlying.asInstanceOf[ActorCell].mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
slow.underlying.asInstanceOf[ActorCell].mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
fast.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
fast.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount must be >
(slow.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount)
system.stop(slow)
system.stop(fast)
}

View file

@ -1,14 +1,17 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.dispatch
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import java.util.concurrent.{ TimeUnit, BlockingQueue }
import java.util.concurrent.ConcurrentLinkedQueue
import akka.util._
import akka.util.duration._
import akka.testkit.AkkaSpec
import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef }
import java.util.concurrent.{ ConcurrentLinkedQueue, BlockingQueue }
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll }
import com.typesafe.config.Config
import akka.actor.ActorSystem
import akka.actor.{ RepointableRef, Props, DeadLetter, ActorSystem, ActorRefWithCell, ActorRef, ActorCell }
import akka.testkit.AkkaSpec
import akka.util.duration.intToDurationInt
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
@ -39,9 +42,10 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
q.numberOfMessages must be === config.capacity
q.hasMessages must be === true
intercept[MessageQueueAppendFailedException] {
q.enqueue(null, exampleMessage)
}
system.eventStream.subscribe(testActor, classOf[DeadLetter])
q.enqueue(testActor, exampleMessage)
expectMsg(DeadLetter(exampleMessage.message, system.deadLetters, testActor))
system.eventStream.unsubscribe(testActor, classOf[DeadLetter])
q.dequeue must be === exampleMessage
q.numberOfMessages must be(config.capacity - 1)
@ -75,7 +79,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
result
}
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)(system)
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters, system)
def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) {
q must not be null
@ -136,8 +140,8 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
class DefaultMailboxSpec extends MailboxSpec {
lazy val name = "The default mailbox implementation"
def factory = {
case u: UnboundedMailbox u.create(None)
case b: BoundedMailbox b.create(None)
case u: UnboundedMailbox u.create(None, None)
case b: BoundedMailbox b.create(None, None)
}
}
@ -145,8 +149,8 @@ class PriorityMailboxSpec extends MailboxSpec {
val comparator = PriorityGenerator(_.##)
lazy val name = "The priority mailbox implementation"
def factory = {
case UnboundedMailbox() new UnboundedPriorityMailbox(comparator).create(None)
case BoundedMailbox(capacity, pushTimeOut) new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None)
case UnboundedMailbox() new UnboundedPriorityMailbox(comparator).create(None, None)
case BoundedMailbox(capacity, pushTimeOut) new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None, None)
}
}
@ -158,13 +162,13 @@ object CustomMailboxSpec {
"""
class MyMailboxType(settings: ActorSystem.Settings, config: Config) extends MailboxType {
override def create(owner: Option[ActorContext]) = owner match {
override def create(owner: Option[ActorRef], system: Option[ActorSystem]) = owner match {
case Some(o) new MyMailbox(o)
case None throw new Exception("no mailbox owner given")
}
}
class MyMailbox(owner: ActorContext) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
class MyMailbox(owner: ActorRef) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
final val queue = new ConcurrentLinkedQueue[Envelope]()
}
}
@ -174,7 +178,11 @@ class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) {
"Dispatcher configuration" must {
"support custom mailboxType" in {
val actor = system.actorOf(Props.empty.withDispatcher("my-dispatcher"))
val queue = actor.asInstanceOf[LocalActorRef].underlying.mailbox.messageQueue
awaitCond(actor match {
case r: RepointableRef r.isStarted
case _ true
}, 1 second, 10 millis)
val queue = actor.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox.messageQueue
queue.getClass must be(classOf[CustomMailboxSpec.MyMailbox])
}
}

View file

@ -1,12 +1,14 @@
package akka.dispatch
import akka.actor.{ Props, LocalActorRef, Actor }
import akka.testkit.AkkaSpec
import akka.pattern.ask
import akka.util.duration._
import akka.testkit.DefaultTimeout
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import com.typesafe.config.Config
import akka.actor.ActorSystem
import akka.actor.{ Props, InternalActorRef, ActorSystem, Actor }
import akka.pattern.ask
import akka.testkit.{ DefaultTimeout, AkkaSpec }
import akka.util.duration.intToDurationInt
object PriorityDispatcherSpec {
val config = """
@ -54,7 +56,7 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
case i: Int acc = i :: acc
case 'Result sender.tell(acc)
}
}).withDispatcher(dispatcherKey)).asInstanceOf[LocalActorRef]
}).withDispatcher(dispatcherKey)).asInstanceOf[InternalActorRef]
actor.suspend //Make sure the actor isn't treating any messages, let it buffer the incoming messages

View file

@ -74,6 +74,17 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
}
}
"not allow null as subscriber" in {
val bus = new EventStream(true)
intercept[IllegalArgumentException] { bus.subscribe(null, classOf[M]) }.getMessage must be("subscriber is null")
}
"not allow null as unsubscriber" in {
val bus = new EventStream(true)
intercept[IllegalArgumentException] { bus.unsubscribe(null, classOf[M]) }.getMessage must be("subscriber is null")
intercept[IllegalArgumentException] { bus.unsubscribe(null) }.getMessage must be("subscriber is null")
}
"be able to log unhandled messages" in {
val sys = ActorSystem("EventStreamSpecUnhandled", configUnhandled)
try {

View file

@ -0,0 +1,121 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.pattern
import akka.testkit._
import akka.util.duration._
import org.scalatest.BeforeAndAfter
import akka.dispatch.{ Promise, Await, Future }
class CircuitBreakerMTSpec extends AkkaSpec with BeforeAndAfter {
@volatile
var breakers: BreakerState = null
class BreakerState {
val halfOpenLatch = new TestLatch(1)
val breaker = new CircuitBreaker(system.scheduler, 5, 100.millis.dilated, 500.millis.dilated)
.onHalfOpen(halfOpenLatch.countDown())
}
before {
breakers = new BreakerState()
}
def unreliableCall(param: String) = {
param match {
case "fail" throw new RuntimeException("FAIL")
case _ param
}
}
def openBreaker: Unit = {
for (i 1 to 5)
Await.result(breakers.breaker.withCircuitBreaker(Future(unreliableCall("fail"))) recoverWith {
case _ Promise.successful("OK")
}, 1.second.dilated)
}
"A circuit breaker being called by many threads" must {
"allow many calls while in closed state with no errors" in {
val futures = for (i 1 to 100) yield breakers.breaker.withCircuitBreaker(Future { Thread.sleep(10); unreliableCall("succeed") })
val futureList = Future.sequence(futures)
val result = Await.result(futureList, 1.second.dilated)
result.size must be(100)
result.distinct.size must be(1)
result.distinct must contain("succeed")
}
"transition to open state upon reaching failure limit and fail-fast" in {
openBreaker
val futures = for (i 1 to 100) yield breakers.breaker.withCircuitBreaker(Future {
Thread.sleep(10); unreliableCall("success")
}) recoverWith {
case _: CircuitBreakerOpenException Promise.successful("CBO")
}
val futureList = Future.sequence(futures)
val result = Await.result(futureList, 1.second.dilated)
result.size must be(100)
result.distinct.size must be(1)
result.distinct must contain("CBO")
}
"allow a single call through in half-open state" in {
openBreaker
Await.ready(breakers.halfOpenLatch, 2.seconds.dilated)
val futures = for (i 1 to 100) yield breakers.breaker.withCircuitBreaker(Future {
Thread.sleep(10); unreliableCall("succeed")
}) recoverWith {
case _: CircuitBreakerOpenException Promise.successful("CBO")
}
val futureList = Future.sequence(futures)
val result = Await.result(futureList, 1.second.dilated)
result.size must be(100)
result.distinct.size must be(2)
result.distinct must contain("succeed")
result.distinct must contain("CBO")
}
"recover and reset the breaker after the reset timeout" in {
openBreaker
Await.ready(breakers.halfOpenLatch, 2.seconds.dilated)
Await.ready(breakers.breaker.withCircuitBreaker(Future(unreliableCall("succeed"))), 1.second.dilated)
val futures = for (i 1 to 100) yield breakers.breaker.withCircuitBreaker(Future {
Thread.sleep(10); unreliableCall("succeed")
}) recoverWith {
case _: CircuitBreakerOpenException Promise.successful("CBO")
}
val futureList = Future.sequence(futures)
val result = Await.result(futureList, 1.second.dilated)
result.size must be(100)
result.distinct.size must be(1)
result.distinct must contain("succeed")
}
}
}

View file

@ -0,0 +1,243 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.pattern
import akka.util.duration._
import akka.testkit._
import org.scalatest.BeforeAndAfter
import akka.dispatch.Future
import akka.dispatch.Await
object CircuitBreakerSpec {
class TestException extends RuntimeException
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class CircuitBreakerSpec extends AkkaSpec with BeforeAndAfter {
import CircuitBreakerSpec.TestException
val awaitTimeout = 2.seconds.dilated
@volatile
var breakers: TestCircuitBreakers = null
class TestCircuitBreakers {
val halfOpenLatch = new TestLatch(1)
val openLatch = new TestLatch(1)
val closedLatch = new TestLatch(1)
val shortCallTimeoutCb = new CircuitBreaker(system.scheduler, 1, 50.millis.dilated, 500.millis.dilated)
.onClose(closedLatch.countDown())
.onHalfOpen(halfOpenLatch.countDown())
.onOpen(openLatch.countDown())
val shortResetTimeoutCb = new CircuitBreaker(system.scheduler, 1, 1000.millis.dilated, 50.millis.dilated)
.onClose(closedLatch.countDown())
.onHalfOpen(halfOpenLatch.countDown())
.onOpen(openLatch.countDown())
val longCallTimeoutCb = new CircuitBreaker(system.scheduler, 1, 5 seconds, 500.millis.dilated)
.onClose(closedLatch.countDown())
.onHalfOpen(halfOpenLatch.countDown())
.onOpen(openLatch.countDown())
val longResetTimeoutCb = new CircuitBreaker(system.scheduler, 1, 100.millis.dilated, 5 seconds)
.onClose(closedLatch.countDown())
.onHalfOpen(halfOpenLatch.countDown())
.onOpen(openLatch.countDown())
val multiFailureCb = new CircuitBreaker(system.scheduler, 5, 200.millis.dilated, 500.millis.dilated)
.onClose(closedLatch.countDown())
.onHalfOpen(halfOpenLatch.countDown())
.onOpen(openLatch.countDown())
}
before {
breakers = new TestCircuitBreakers
}
def checkLatch(latch: TestLatch) {
Await.ready(latch, awaitTimeout)
}
def throwException = throw new TestException
def sayHi = "hi"
"A synchronous circuit breaker that is open" must {
"throw exceptions when called before reset timeout" in {
intercept[TestException] {
breakers.longResetTimeoutCb.withSyncCircuitBreaker(throwException)
}
checkLatch(breakers.openLatch)
intercept[CircuitBreakerOpenException] {
breakers.longResetTimeoutCb.withSyncCircuitBreaker(sayHi)
}
}
"transition to half-open on reset timeout" in {
intercept[TestException] {
breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException)
}
checkLatch(breakers.halfOpenLatch)
}
}
"A synchronous circuit breaker that is half-open" must {
"pass through next call and close on success" in {
intercept[TestException] {
breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException)
}
checkLatch(breakers.halfOpenLatch)
assert("hi" == breakers.shortResetTimeoutCb.withSyncCircuitBreaker(sayHi))
checkLatch(breakers.closedLatch)
}
"open on exception in call" in {
intercept[TestException] {
breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException)
}
checkLatch(breakers.halfOpenLatch)
intercept[TestException] {
breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException)
}
checkLatch(breakers.openLatch)
}
}
"A synchronous circuit breaker that is closed" must {
"allow calls through" in {
breakers.longCallTimeoutCb.withSyncCircuitBreaker(sayHi) must be("hi")
}
"increment failure count on failure" in {
intercept[TestException] {
breakers.longCallTimeoutCb.withSyncCircuitBreaker(throwException)
}
checkLatch(breakers.openLatch)
breakers.longCallTimeoutCb.currentFailureCount must be(1)
}
"reset failure count after success" in {
intercept[TestException] {
breakers.multiFailureCb.withSyncCircuitBreaker(throwException)
}
breakers.multiFailureCb.currentFailureCount must be(1)
breakers.multiFailureCb.withSyncCircuitBreaker(sayHi)
breakers.multiFailureCb.currentFailureCount must be(0)
}
"increment failure count on callTimeout" in {
breakers.shortCallTimeoutCb.withSyncCircuitBreaker({
100.millis.dilated.sleep()
})
breakers.shortCallTimeoutCb.currentFailureCount must be(1)
}
}
"An asynchronous circuit breaker that is open" must {
"throw exceptions when called before reset timeout" in {
breakers.longResetTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.openLatch)
intercept[CircuitBreakerOpenException] {
Await.result(
breakers.longResetTimeoutCb.withCircuitBreaker(Future(sayHi)),
awaitTimeout)
}
}
"transition to half-open on reset timeout" in {
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.halfOpenLatch)
}
}
"An asynchronous circuit breaker that is half-open" must {
"pass through next call and close on success" in {
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.halfOpenLatch)
Await.result(
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(sayHi)),
awaitTimeout) must be("hi")
checkLatch(breakers.closedLatch)
}
"re-open on exception in call" in {
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.halfOpenLatch)
intercept[TestException] {
Await.result(
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)),
awaitTimeout)
}
checkLatch(breakers.openLatch)
}
"re-open on async failure" in {
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.halfOpenLatch)
breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.openLatch)
}
}
"An asynchronous circuit breaker that is closed" must {
"allow calls through" in {
Await.result(
breakers.longCallTimeoutCb.withCircuitBreaker(Future(sayHi)),
awaitTimeout) must be("hi")
}
"increment failure count on exception" in {
intercept[TestException] {
Await.result(
breakers.longCallTimeoutCb.withCircuitBreaker(Future(throwException)),
awaitTimeout)
}
checkLatch(breakers.openLatch)
breakers.longCallTimeoutCb.currentFailureCount must be(1)
}
"increment failure count on async failure" in {
breakers.longCallTimeoutCb.withCircuitBreaker(Future(throwException))
checkLatch(breakers.openLatch)
breakers.longCallTimeoutCb.currentFailureCount must be(1)
}
"reset failure count after success" in {
breakers.multiFailureCb.withCircuitBreaker(Future(sayHi))
val latch = TestLatch(4)
for (n 1 to 4) breakers.multiFailureCb.withCircuitBreaker(Future(throwException))
awaitCond(breakers.multiFailureCb.currentFailureCount == 4, awaitTimeout)
breakers.multiFailureCb.withCircuitBreaker(Future(sayHi))
awaitCond(breakers.multiFailureCb.currentFailureCount == 0, awaitTimeout)
}
"increment failure count on callTimeout" in {
breakers.shortCallTimeoutCb.withCircuitBreaker {
Future {
100.millis.dilated.sleep()
sayHi
}
}
checkLatch(breakers.openLatch)
breakers.shortCallTimeoutCb.currentFailureCount must be(1)
}
}
}

View file

@ -4,15 +4,14 @@
package akka.routing
import java.util.concurrent.atomic.AtomicInteger
import org.junit.runner.RunWith
import akka.actor.{ Props, LocalActorRef, Deploy, Actor, ActorRef }
import akka.actor.{ Props, Deploy, Actor, ActorRef }
import akka.ConfigurationException
import akka.dispatch.Await
import akka.pattern.{ ask, gracefulStop }
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
import akka.util.duration.intToDurationInt
import akka.actor.UnstartedCell
object ConfiguredLocalRoutingSpec {
val config = """
@ -47,6 +46,14 @@ object ConfiguredLocalRoutingSpec {
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender {
def routerConfig(ref: ActorRef): RouterConfig = ref match {
case r: RoutedActorRef
r.underlying match {
case c: RoutedActorCell c.routerConfig
case _: UnstartedCell awaitCond(r.isStarted, 1 second, 10 millis); routerConfig(ref)
}
}
"RouterConfig" must {
"be picked up from Props" in {
@ -55,7 +62,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
case "get" sender ! context.props
}
}).withRouter(RoundRobinRouter(12)), "someOther")
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RoundRobinRouter(12)
routerConfig(actor) must be === RoundRobinRouter(12)
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
}
@ -65,7 +72,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
case "get" sender ! context.props
}
}).withRouter(RoundRobinRouter(12)), "config")
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RandomRouter(4)
routerConfig(actor) must be === RandomRouter(4)
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
}
@ -75,7 +82,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
case "get" sender ! context.props
}
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "someOther")
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RoundRobinRouter(12)
routerConfig(actor) must be === RoundRobinRouter(12)
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
}
@ -85,7 +92,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
case "get" sender ! context.props
}
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "config")
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RandomRouter(4)
routerConfig(actor) must be === RandomRouter(4)
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
}

View file

@ -12,10 +12,11 @@ import akka.dispatch.Await
import akka.util.Duration
import akka.ConfigurationException
import com.typesafe.config.ConfigFactory
import akka.pattern.ask
import akka.pattern.{ ask, pipe }
import java.util.concurrent.ConcurrentHashMap
import com.typesafe.config.Config
import akka.dispatch.Dispatchers
import akka.util.Timeout
object RoutingSpec {
@ -73,7 +74,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
watch(router)
watch(c2)
system.stop(c2)
expectMsg(Terminated(c2))
expectMsgPF() {
case t @ Terminated(`c2`) if t.existenceConfirmed == true t
}
// it might take a while until the Router has actually processed the Terminated message
awaitCond {
router ! ""
@ -84,7 +87,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
res == Seq(c1, c1)
}
system.stop(c1)
expectMsg(Terminated(router))
expectMsgPF() {
case t @ Terminated(`router`) if t.existenceConfirmed == true t
}
}
"be able to send their routees" in {
@ -167,6 +172,18 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
expectMsg("restarted")
}
"must start in-line for context.actorOf()" in {
system.actorOf(Props(new Actor {
def receive = {
case "start"
context.actorOf(Props(new Actor {
def receive = { case x sender ! x }
}).withRouter(RoundRobinRouter(2))) ? "hello" pipeTo sender
}
})) ! "start"
expectMsg("hello")
}
}
"no router" must {
@ -524,7 +541,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
}
}
"support custom router" in {
val myrouter = system.actorOf(Props().withRouter(FromConfig), "myrouter")
val myrouter = system.actorOf(Props.empty.withRouter(FromConfig), "myrouter")
myrouter.isTerminated must be(false)
}
}
@ -536,7 +553,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
}
"count votes as intended - not as in Florida" in {
val routedActor = system.actorOf(Props().withRouter(VoteCountRouter()))
val routedActor = system.actorOf(Props.empty.withRouter(VoteCountRouter()))
routedActor ! DemocratVote
routedActor ! DemocratVote
routedActor ! RepublicanVote

View file

@ -0,0 +1,23 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor;
import akka.util.Unsafe;
final class AbstractActorCell {
final static long mailboxOffset;
final static long childrenOffset;
final static long nextNameOffset;
static {
try {
mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_mailboxDoNotCallMeDirectly"));
childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_childrenRefsDoNotCallMeDirectly"));
nextNameOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_nextNameDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}
}
}

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor;
import akka.util.Unsafe;
final class AbstractActorRef {
final static long cellOffset;
static {
try {
cellOffset = Unsafe.instance.objectFieldOffset(RepointableActorRef.class.getDeclaredField("_cellDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}
}
}

View file

@ -0,0 +1,18 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.pattern;
import akka.util.Unsafe;
class AbstractCircuitBreaker {
protected final static long stateOffset;
static {
try {
stateOffset = Unsafe.instance.objectFieldOffset(CircuitBreaker.class.getDeclaredField("_currentStateDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}
}
}

View file

@ -8,10 +8,12 @@ import akka.util.Unsafe;
final class AbstractPromiseActorRef {
final static long stateOffset;
final static long watchedByOffset;
static {
try {
stateOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_stateDoNotCallMeDirectly"));
watchedByOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_watchedByDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}

View file

@ -9,7 +9,6 @@ package akka
* <ul>
* <li>a uuid for tracking purposes</li>
* <li>toString that includes exception name, message and uuid</li>
* <li>toLongString which also includes the stack trace</li>
* </ul>
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed

View file

@ -7,7 +7,6 @@ package akka.actor
import akka.AkkaException
import scala.reflect.BeanProperty
import scala.util.control.NoStackTrace
import scala.collection.immutable.Stack
import java.util.regex.Pattern
/**
@ -59,7 +58,7 @@ case object Kill extends Kill {
/**
* When Death Watch is used, the watcher will receive a Terminated(watched) message when watched is terminated.
*/
case class Terminated(@BeanProperty actor: ActorRef) extends PossiblyHarmful
case class Terminated(@BeanProperty actor: ActorRef)(@BeanProperty val existenceConfirmed: Boolean) extends AutoReceivedMessage
abstract class ReceiveTimeout extends PossiblyHarmful
@ -135,8 +134,7 @@ class ActorInitializationException private[akka] (actor: ActorRef, message: Stri
* there might be more of them in the future, or not.
*/
class InvalidMessageException private[akka] (message: String, cause: Throwable = null)
extends AkkaException(message, cause)
with NoStackTrace {
extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null)
}
@ -279,18 +277,14 @@ trait Actor {
*/
protected[akka] implicit val context: ActorContext = {
val contextStack = ActorCell.contextStack.get
def noContextError =
if ((contextStack.isEmpty) || (contextStack.head eq null))
throw new ActorInitializationException(
"\n\tYou cannot create an instance of [" + getClass.getName + "] explicitly using the constructor (new)." +
"\n\tYou have to use one of the factory methods to create a new actor. Either use:" +
"\n\t\t'val actor = context.actorOf(Props[MyActor])' (to create a supervised child actor from within an actor), or" +
"\n\t\t'val actor = system.actorOf(Props(new MyActor(..)))' (to create a top level actor from the ActorSystem)")
if (contextStack.isEmpty) noContextError
val c = contextStack.head
if (c eq null) noContextError
ActorCell.contextStack.set(contextStack.push(null))
ActorCell.contextStack.set(null :: contextStack)
c
}

View file

@ -6,16 +6,16 @@ package akka.actor
import akka.dispatch._
import scala.annotation.tailrec
import scala.collection.immutable.{ Stack, TreeMap }
import java.util.concurrent.TimeUnit
import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.event.Logging.{ Debug, Warning, Error }
import akka.util.{ Duration, Helpers }
import akka.japi.Procedure
import java.io.{ NotSerializableException, ObjectOutputStream }
import akka.serialization.SerializationExtension
import akka.util.NonFatal
import akka.event.Logging.LogEventException
import collection.immutable.{ TreeSet, TreeMap }
import akka.util.{ Unsafe, Duration, Helpers, NonFatal }
import java.util.concurrent.atomic.AtomicLong
//TODO: everything here for current compatibility - could be limited more
@ -168,14 +168,86 @@ trait UntypedActorContext extends ActorContext {
}
/**
* INTERNAL API
*/
private[akka] trait Cell {
/**
* The self reference which this Cell is attached to.
*/
def self: ActorRef
/**
* The system within which this Cell lives.
*/
def system: ActorSystem
/**
* The system internals where this Cell lives.
*/
def systemImpl: ActorSystemImpl
/**
* Recursively suspend this actor and all its children.
*/
def suspend(): Unit
/**
* Recursively resume this actor and all its children.
*/
def resume(): Unit
/**
* Restart this actor (will recursively restart or stop all children).
*/
def restart(cause: Throwable): Unit
/**
* Recursively terminate this actor and all its children.
*/
def stop(): Unit
/**
* Returns true if the actor is locally known to be terminated, false if
* alive or uncertain.
*/
def isTerminated: Boolean
/**
* The supervisor of this actor.
*/
def parent: InternalActorRef
/**
* All children of this actor, including only reserved-names.
*/
def childrenRefs: ActorCell.ChildrenContainer
/**
* Enqueue a message to be sent to the actor; may or may not actually
* schedule the actor to run, depending on which type of cell it is.
*/
def tell(message: Any, sender: ActorRef): Unit
/**
* Enqueue a message to be sent to the actor; may or may not actually
* schedule the actor to run, depending on which type of cell it is.
*/
def sendSystemMessage(msg: SystemMessage): Unit
/**
* Returns true if the actor is local, i.e. if it is actually scheduled
* on a Thread in the current JVM when run.
*/
def isLocal: Boolean
/**
* If the actor isLocal, returns whether messages are currently queued,
* false otherwise.
*/
def hasMessages: Boolean
/**
* If the actor isLocal, returns the number of messages currently queued,
* which may be a costly operation, 0 otherwise.
*/
def numberOfMessages: Int
}
/**
* Everything in here is completely Akka PRIVATE. You will not find any
* supported APIs in this place. This is not the API you were looking
* for! (waves hand)
*/
private[akka] object ActorCell {
val contextStack = new ThreadLocal[Stack[ActorContext]] {
override def initialValue = Stack[ActorContext]()
val contextStack = new ThreadLocal[List[ActorContext]] {
override def initialValue: List[ActorContext] = Nil
}
final val emptyCancellable: Cancellable = new Cancellable {
@ -185,7 +257,9 @@ private[akka] object ActorCell {
final val emptyReceiveTimeoutData: (Long, Cancellable) = (-1, emptyCancellable)
final val behaviorStackPlaceHolder: Stack[Actor.Receive] = Stack.empty.push(Actor.emptyBehavior)
final val emptyBehaviorStack: List[Actor.Receive] = Nil
final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty
sealed trait SuspendReason
case object UserRequest extends SuspendReason
@ -200,10 +274,18 @@ private[akka] object ActorCell {
def children: Iterable[ActorRef]
def stats: Iterable[ChildRestartStats]
def shallDie(actor: ActorRef): ChildrenContainer
/**
* reserve that name or throw an exception
*/
def reserve(name: String): ChildrenContainer
/**
* cancel a reservation
*/
def unreserve(name: String): ChildrenContainer
}
trait EmptyChildrenContainer extends ChildrenContainer {
val emptyStats = TreeMap.empty[String, ChildRestartStats]
val emptyStats = TreeMap.empty[String, ChildStats]
def add(child: ActorRef): ChildrenContainer =
new NormalChildrenContainer(emptyStats.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = this
@ -212,6 +294,8 @@ private[akka] object ActorCell {
def children: Iterable[ActorRef] = Nil
def stats: Iterable[ChildRestartStats] = Nil
def shallDie(actor: ActorRef): ChildrenContainer = this
def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved))
def unreserve(name: String): ChildrenContainer = this
override def toString = "no children"
}
@ -227,6 +311,8 @@ private[akka] object ActorCell {
*/
object TerminatedChildrenContainer extends EmptyChildrenContainer {
override def add(child: ActorRef): ChildrenContainer = this
override def reserve(name: String): ChildrenContainer =
throw new IllegalStateException("cannot reserve actor name '" + name + "': already terminated")
}
/**
@ -235,32 +321,46 @@ private[akka] object ActorCell {
* calling context.stop(child) and processing the ChildTerminated() system
* message).
*/
class NormalChildrenContainer(c: TreeMap[String, ChildRestartStats]) extends ChildrenContainer {
class NormalChildrenContainer(c: TreeMap[String, ChildStats]) extends ChildrenContainer {
def add(child: ActorRef): ChildrenContainer = new NormalChildrenContainer(c.updated(child.path.name, ChildRestartStats(child)))
def add(child: ActorRef): ChildrenContainer =
new NormalChildrenContainer(c.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = NormalChildrenContainer(c - child.path.name)
def getByName(name: String): Option[ChildRestartStats] = c get name
def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match {
case c @ Some(crs) if (crs.child == actor) c
def getByName(name: String): Option[ChildRestartStats] = c.get(name) match {
case s @ Some(_: ChildRestartStats) s.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
def children: Iterable[ActorRef] = c.values.view.map(_.child)
def getByRef(actor: ActorRef): Option[ChildRestartStats] = c.get(actor.path.name) match {
case c @ Some(crs: ChildRestartStats) if (crs.child == actor) c.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
def stats: Iterable[ChildRestartStats] = c.values
def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) child }
def stats: Iterable[ChildRestartStats] = c.values.collect { case c: ChildRestartStats c }
def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
def reserve(name: String): ChildrenContainer =
if (c contains name)
throw new InvalidActorNameException("actor name " + name + " is not unique!")
else new NormalChildrenContainer(c.updated(name, ChildNameReserved))
def unreserve(name: String): ChildrenContainer = c.get(name) match {
case Some(ChildNameReserved) NormalChildrenContainer(c - name)
case _ this
}
override def toString =
if (c.size > 20) c.size + " children"
else c.mkString("children:\n ", "\n ", "")
}
object NormalChildrenContainer {
def apply(c: TreeMap[String, ChildRestartStats]): ChildrenContainer =
def apply(c: TreeMap[String, ChildStats]): ChildrenContainer =
if (c.isEmpty) EmptyChildrenContainer
else new NormalChildrenContainer(c)
}
@ -275,7 +375,7 @@ private[akka] object ActorCell {
* type of container, depending on whether or not children are left and whether or not
* the reason was Terminating.
*/
case class TerminatingChildrenContainer(c: TreeMap[String, ChildRestartStats], toDie: Set[ActorRef], reason: SuspendReason)
case class TerminatingChildrenContainer(c: TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
extends ChildrenContainer {
def add(child: ActorRef): ChildrenContainer = copy(c.updated(child.path.name, ChildRestartStats(child)))
@ -289,19 +389,35 @@ private[akka] object ActorCell {
else copy(c - child.path.name, t)
}
def getByName(name: String): Option[ChildRestartStats] = c get name
def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match {
case c @ Some(crs) if (crs.child == actor) c
def getByName(name: String): Option[ChildRestartStats] = c.get(name) match {
case s @ Some(_: ChildRestartStats) s.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
def children: Iterable[ActorRef] = c.values.view.map(_.child)
def getByRef(actor: ActorRef): Option[ChildRestartStats] = c.get(actor.path.name) match {
case c @ Some(crs: ChildRestartStats) if (crs.child == actor) c.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
def stats: Iterable[ChildRestartStats] = c.values
def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) child }
def stats: Iterable[ChildRestartStats] = c.values.collect { case c: ChildRestartStats c }
def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)
def reserve(name: String): ChildrenContainer = reason match {
case Termination throw new IllegalStateException("cannot reserve actor name '" + name + "': terminating")
case _
if (c contains name)
throw new InvalidActorNameException("actor name " + name + " is not unique!")
else copy(c = c.updated(name, ChildNameReserved))
}
def unreserve(name: String): ChildrenContainer = c.get(name) match {
case Some(ChildNameReserved) copy(c = c - name)
case _ this
}
override def toString =
if (c.size > 20) c.size + " children"
else c.mkString("children (" + toDie.size + " terminating):\n ", "\n ", "\n") + toDie
@ -315,11 +431,13 @@ private[akka] class ActorCell(
val system: ActorSystemImpl,
val self: InternalActorRef,
val props: Props,
@volatile var parent: InternalActorRef,
/*no member*/ _receiveTimeout: Option[Duration]) extends UntypedActorContext {
@volatile var parent: InternalActorRef) extends UntypedActorContext with Cell {
import AbstractActorCell.{ mailboxOffset, childrenOffset, nextNameOffset }
import ActorCell._
final def isLocal = true
final def systemImpl = system
protected final def guardian = self
@ -350,11 +468,49 @@ private[akka] class ActorCell(
/**
* In milliseconds
*/
var receiveTimeoutData: (Long, Cancellable) =
if (_receiveTimeout.isDefined) (_receiveTimeout.get.toMillis, emptyCancellable) else emptyReceiveTimeoutData
var receiveTimeoutData: (Long, Cancellable) = emptyReceiveTimeoutData
@volatile
var childrenRefs: ChildrenContainer = EmptyChildrenContainer
private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer
def childrenRefs: ChildrenContainer = Unsafe.instance.getObjectVolatile(this, childrenOffset).asInstanceOf[ChildrenContainer]
private def swapChildrenRefs(oldChildren: ChildrenContainer, newChildren: ChildrenContainer): Boolean =
Unsafe.instance.compareAndSwapObject(this, childrenOffset, oldChildren, newChildren)
@tailrec private def reserveChild(name: String): Boolean = {
val c = childrenRefs
swapChildrenRefs(c, c.reserve(name)) || reserveChild(name)
}
@tailrec private def unreserveChild(name: String): Boolean = {
val c = childrenRefs
swapChildrenRefs(c, c.unreserve(name)) || unreserveChild(name)
}
@tailrec private def addChild(ref: ActorRef): Boolean = {
val c = childrenRefs
swapChildrenRefs(c, c.add(ref)) || addChild(ref)
}
@tailrec private def shallDie(ref: ActorRef): Boolean = {
val c = childrenRefs
swapChildrenRefs(c, c.shallDie(ref)) || shallDie(ref)
}
@tailrec private def removeChild(ref: ActorRef): ChildrenContainer = {
val c = childrenRefs
val n = c.remove(ref)
if (swapChildrenRefs(c, n)) n
else removeChild(ref)
}
@tailrec private def setChildrenTerminationReason(reason: SuspendReason): Boolean = {
childrenRefs match {
case c: TerminatingChildrenContainer swapChildrenRefs(c, c.copy(reason = reason)) || setChildrenTerminationReason(reason)
case _ false
}
}
private def isTerminating = childrenRefs match {
case TerminatingChildrenContainer(_, _, Termination) true
@ -366,7 +522,7 @@ private[akka] class ActorCell(
case _ true
}
private def _actorOf(props: Props, name: String): ActorRef = {
private def _actorOf(props: Props, name: String, async: Boolean): ActorRef = {
if (system.settings.SerializeAllCreators && !props.creator.isInstanceOf[NoSerializationVerificationNeeded]) {
val ser = SerializationExtension(system)
ser.serialize(props.creator) match {
@ -377,56 +533,98 @@ private[akka] class ActorCell(
}
}
}
// in case we are currently terminating, swallow creation requests and return EmptyLocalActorRef
if (isTerminating) provider.actorFor(self, Seq(name))
/*
* in case we are currently terminating, fail external attachChild requests
* (internal calls cannot happen anyway because we are suspended)
*/
if (isTerminating) throw new IllegalStateException("cannot create children while terminating or terminated")
else {
val actor = provider.actorOf(systemImpl, props, self, self.path / name, false, None, true)
childrenRefs = childrenRefs.add(actor)
reserveChild(name)
// this name will either be unreserved or overwritten with a real child below
val actor =
try {
provider.actorOf(systemImpl, props, self, self.path / name,
systemService = false, deploy = None, lookupDeploy = true, async = async)
} catch {
case NonFatal(e)
unreserveChild(name)
throw e
}
addChild(actor)
actor
}
}
def actorOf(props: Props): ActorRef = _actorOf(props, randomName())
def actorOf(props: Props): ActorRef = _actorOf(props, randomName(), async = false)
def actorOf(props: Props, name: String): ActorRef = {
def actorOf(props: Props, name: String): ActorRef = _actorOf(props, checkName(name), async = false)
private def checkName(name: String): String = {
import ActorPath.ElementRegex
name match {
case null throw new InvalidActorNameException("actor name must not be null")
case "" throw new InvalidActorNameException("actor name must not be empty")
case ElementRegex() // this is fine
case ElementRegex() name
case _ throw new InvalidActorNameException("illegal actor name '" + name + "', must conform to " + ElementRegex)
}
childrenRefs.getByName(name) match {
case None _actorOf(props, name)
case _ throw new InvalidActorNameException("actor name " + name + " is not unique!")
}
}
private[akka] def attachChild(props: Props, name: String): ActorRef =
_actorOf(props, checkName(name), async = true)
private[akka] def attachChild(props: Props): ActorRef =
_actorOf(props, randomName(), async = true)
final def stop(actor: ActorRef): Unit = {
if (childrenRefs.getByRef(actor).isDefined) childrenRefs = childrenRefs.shallDie(actor)
val started = actor match {
case r: RepointableRef r.isStarted
case _ true
}
if (childrenRefs.getByRef(actor).isDefined && started) shallDie(actor)
actor.asInstanceOf[InternalActorRef].stop()
}
var currentMessage: Envelope = null
var currentMessage: Envelope = _
var actor: Actor = _
private var behaviorStack: List[Actor.Receive] = emptyBehaviorStack
var watching: Set[ActorRef] = emptyActorRefSet
var watchedBy: Set[ActorRef] = emptyActorRefSet
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
@volatile //This must be volatile since it isn't protected by the mailbox status
var mailbox: Mailbox = _
var nextNameSequence: Long = 0
//Not thread safe, so should only be used inside the actor that inhabits this ActorCell
@volatile private var _nextNameDoNotCallMeDirectly = 0L
final protected def randomName(): String = {
val n = nextNameSequence
nextNameSequence = n + 1
Helpers.base64(n)
@tailrec def inc(): Long = {
val current = Unsafe.instance.getLongVolatile(this, nextNameOffset)
if (Unsafe.instance.compareAndSwapLong(this, nextNameOffset, current, current + 1)) current
else inc()
}
Helpers.base64(inc())
}
@inline
final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
@volatile private var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
/**
* INTERNAL API
*
* Returns a reference to the current mailbox
*/
@inline final def mailbox: Mailbox = Unsafe.instance.getObjectVolatile(this, mailboxOffset).asInstanceOf[Mailbox]
/**
* INTERNAL API
*
* replaces the current mailbox using getAndSet semantics
*/
@tailrec final def swapMailbox(newMailbox: Mailbox): Mailbox = {
val oldMailbox = mailbox
if (!Unsafe.instance.compareAndSwapObject(this, mailboxOffset, oldMailbox, newMailbox)) swapMailbox(newMailbox)
else oldMailbox
}
final def hasMessages: Boolean = mailbox.hasMessages
final def numberOfMessages: Int = mailbox.numberOfMessages
val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
/**
* UntypedActorContext impl
@ -435,20 +633,22 @@ private[akka] class ActorCell(
final def isTerminated: Boolean = mailbox.isClosed
final def start(): Unit = {
final def start(): this.type = {
/*
* Create the mailbox and enqueue the Create() message to ensure that
* this is processed before anything else.
*/
mailbox = dispatcher.createMailbox(this)
swapMailbox(dispatcher.createMailbox(this))
mailbox.setActor(this)
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
mailbox.systemEnqueue(self, Create())
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
parent.sendSystemMessage(akka.dispatch.Supervise(self))
// This call is expected to start off the actor by scheduling its mailbox.
dispatcher.attach(this)
this
}
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
@ -460,16 +660,22 @@ private[akka] class ActorCell(
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
final def stop(): Unit = dispatcher.systemDispatch(this, Terminate())
override final def watch(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Link(subject))
subject
override final def watch(subject: ActorRef): ActorRef = subject match {
case a: InternalActorRef
if (a != self && !watching.contains(a)) {
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
watching += a
}
a
}
override final def unwatch(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Unlink(subject))
subject
override final def unwatch(subject: ActorRef): ActorRef = subject match {
case a: InternalActorRef
if (a != self && watching.contains(a)) {
a.sendSystemMessage(Unwatch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
watching -= a
}
a
}
final def children: Iterable[ActorRef] = childrenRefs.children
@ -477,13 +683,13 @@ private[akka] class ActorCell(
/**
* Impl UntypedActorContext
*/
final def getChildren(): java.lang.Iterable[ActorRef] = {
import scala.collection.JavaConverters.asJavaIterableConverter
asJavaIterableConverter(children).asJava
}
final def getChildren(): java.lang.Iterable[ActorRef] =
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
final def tell(message: Any, sender: ActorRef): Unit =
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)(system))
def tell(message: Any, sender: ActorRef): Unit =
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender, system))
override def sendSystemMessage(message: SystemMessage): Unit = dispatcher.systemDispatch(this, message)
final def sender: ActorRef = currentMessage match {
case null system.deadLetters
@ -493,25 +699,21 @@ private[akka] class ActorCell(
//This method is in charge of setting up the contextStack and create a new instance of the Actor
protected def newActor(): Actor = {
contextStack.set(contextStack.get.push(this))
contextStack.set(this :: contextStack.get)
try {
import ActorCell.behaviorStackPlaceHolder
behaviorStack = behaviorStackPlaceHolder
behaviorStack = emptyBehaviorStack
val instance = props.creator.apply()
if (instance eq null)
throw new ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'")
behaviorStack = behaviorStack match {
case `behaviorStackPlaceHolder` Stack.empty.push(instance.receive)
case newBehaviors Stack.empty.push(instance.receive).pushAll(newBehaviors.reverse.drop(1))
}
// If no becomes were issued, the actors behavior is its receive method
behaviorStack = if (behaviorStack.isEmpty) instance.receive :: behaviorStack else behaviorStack
instance
} finally {
val stackAfter = contextStack.get
if (stackAfter.nonEmpty)
contextStack.set(if (stackAfter.head eq null) stackAfter.pop.pop else stackAfter.pop) // pop null marker plus our context
contextStack.set(if (stackAfter.head eq null) stackAfter.tail.tail else stackAfter.tail) // pop null marker plus our context
}
}
@ -550,7 +752,7 @@ private[akka] class ActorCell(
}
childrenRefs match {
case ct: TerminatingChildrenContainer
childrenRefs = ct.copy(reason = Recreation(cause))
setChildrenTerminationReason(Recreation(cause))
dispatcher suspend this
case _
doRecreate(cause, failedActor)
@ -567,15 +769,35 @@ private[akka] class ActorCell(
def resume(): Unit = if (isNormal) dispatcher resume this
def link(subject: ActorRef): Unit = if (!isTerminating) {
if (system.deathWatch.subscribe(self, subject)) {
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + subject))
def addWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
val watcheeSelf = watchee == self
val watcherSelf = watcher == self
if (watcheeSelf && !watcherSelf) {
if (!watchedBy.contains(watcher)) {
watchedBy += watcher
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + watcher))
}
} else if (!watcheeSelf && watcherSelf) {
watch(watchee)
} else {
system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, self)))
}
}
def unlink(subject: ActorRef): Unit = if (!isTerminating) {
if (system.deathWatch.unsubscribe(self, subject)) {
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + subject))
def remWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
val watcheeSelf = watchee == self
val watcherSelf = watcher == self
if (watcheeSelf && !watcherSelf) {
if (watchedBy.contains(watcher)) {
watchedBy -= watcher
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + watcher))
}
} else if (!watcheeSelf && watcherSelf) {
unwatch(watchee)
} else {
system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, self)))
}
}
@ -588,7 +810,7 @@ private[akka] class ActorCell(
childrenRefs match {
case ct: TerminatingChildrenContainer
childrenRefs = ct.copy(reason = Termination)
setChildrenTerminationReason(Termination)
// do not process normal messages while waiting for all children to terminate
dispatcher suspend this
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopping"))
@ -597,7 +819,8 @@ private[akka] class ActorCell(
}
def supervise(child: ActorRef): Unit = if (!isTerminating) {
if (childrenRefs.getByRef(child).isEmpty) childrenRefs = childrenRefs.add(child)
if (childrenRefs.getByRef(child).isEmpty) addChild(child)
handleSupervise(child)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
}
@ -605,13 +828,14 @@ private[akka] class ActorCell(
message match {
case Create() create()
case Recreate(cause) recreate(cause)
case Link(subject) link(subject)
case Unlink(subject) unlink(subject)
case Watch(watchee, watcher) addWatcher(watchee, watcher)
case Unwatch(watchee, watcher) remWatcher(watchee, watcher)
case Suspend() suspend()
case Resume() resume()
case Terminate() terminate()
case Supervise(child) supervise(child)
case ChildTerminated(child) handleChildTerminated(child)
case NoMessage // only here to suppress warning
}
} catch {
case e @ (_: InterruptedException | NonFatal(_)) handleInvokeFailure(e, "error while processing " + message)
@ -633,7 +857,7 @@ private[akka] class ActorCell(
checkReceiveTimeout // Reschedule receive timeout
}
private final def handleInvokeFailure(t: Throwable, message: String): Unit = try {
final def handleInvokeFailure(t: Throwable, message: String): Unit = try {
dispatcher.reportFailure(new LogEventException(Error(t, self.path.toString, clazz(actor), message), t))
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
@ -645,10 +869,8 @@ private[akka] class ActorCell(
}
}
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
if (discardOld) unbecome()
behaviorStack = behaviorStack.push(behavior)
}
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit =
behaviorStack = behavior :: (if (discardOld && behaviorStack.nonEmpty) behaviorStack.tail else behaviorStack)
/**
* UntypedActorContext impl
@ -663,8 +885,9 @@ private[akka] class ActorCell(
def unbecome(): Unit = {
val original = behaviorStack
val popped = original.pop
behaviorStack = if (popped.isEmpty) original else popped
behaviorStack =
if (original.isEmpty || original.tail.isEmpty) actor.receive :: emptyBehaviorStack
else original.tail
}
def autoReceiveMessage(msg: Envelope): Unit = {
@ -673,6 +896,7 @@ private[akka] class ActorCell(
msg.message match {
case Failed(cause) handleFailure(sender, cause)
case t: Terminated watching -= t.actor; receiveMessage(t)
case Kill throw new ActorKilledException("Kill")
case PoisonPill self.stop()
case SelectParent(m) parent.tell(m, msg.sender)
@ -698,11 +922,32 @@ private[akka] class ActorCell(
} finally {
try {
parent.sendSystemMessage(ChildTerminated(self))
system.deathWatch.publish(Terminated(self))
if (!watchedBy.isEmpty) {
val terminated = Terminated(self)(existenceConfirmed = true)
try {
watchedBy foreach {
watcher
try watcher.tell(terminated, self) catch {
case NonFatal(t) system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch"))
}
}
} finally watchedBy = emptyActorRefSet
}
if (!watching.isEmpty) {
try {
watching foreach { // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
case watchee: InternalActorRef try watchee.sendSystemMessage(Unwatch(watchee, self)) catch {
case NonFatal(t) system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch"))
}
}
} finally watching = emptyActorRefSet
}
if (system.settings.DebugLifecycle)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped"))
system.eventStream.publish(Debug(self.path.toString, clazz(a), "stopped"))
} finally {
behaviorStack = ActorCell.behaviorStackPlaceHolder
behaviorStack = emptyBehaviorStack
clearActorFields(a)
actor = null
}
@ -740,8 +985,7 @@ private[akka] class ActorCell(
final def handleChildTerminated(child: ActorRef): Unit = try {
childrenRefs match {
case tc @ TerminatingChildrenContainer(_, _, reason)
val n = tc.remove(child)
childrenRefs = n
val n = removeChild(child)
actor.supervisorStrategy.handleChildTerminated(this, child, children)
if (!n.isInstanceOf[TerminatingChildrenContainer]) reason match {
case Recreation(cause) doRecreate(cause, actor) // doRecreate since this is the continuation of "recreate"
@ -749,7 +993,7 @@ private[akka] class ActorCell(
case _
}
case _
childrenRefs = childrenRefs.remove(child)
removeChild(child)
actor.supervisorStrategy.handleChildTerminated(this, child, children)
}
} catch {
@ -762,6 +1006,11 @@ private[akka] class ActorCell(
}
}
protected def handleSupervise(child: ActorRef): Unit = child match {
case r: RepointableActorRef r.activate()
case _
}
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause))

View file

@ -192,7 +192,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
// TODO RK investigate Phils hash from scala.collection.mutable.HashTable.improve
override def hashCode: Int = {
import scala.util.MurmurHash._
import akka.routing.MurmurHash._
@tailrec
def rec(p: ActorPath, h: Int, c: Int, k: Int): Int = p match {

View file

@ -163,10 +163,24 @@ private[akka] trait ActorRefScope {
def isLocal: Boolean
}
/**
* Refs which are statically known to be local inherit from this Scope
*/
private[akka] trait LocalRef extends ActorRefScope {
final def isLocal = true
}
/**
* RepointableActorRef (and potentially others) may change their locality at
* runtime, meaning that isLocal might not be stable. RepointableActorRef has
* the feature that it starts out not fully started (but you can send to it),
* which is why `isStarted` features here; it is not improbable that cluster
* actor refs will have the same behavior.
*/
private[akka] trait RepointableRef extends ActorRefScope {
def isStarted: Boolean
}
/**
* Internal trait for assembling all the functionality needed internally on
* ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE!
@ -210,6 +224,16 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe
def isLocal: Boolean
}
/**
* Common trait of all actor refs which actually have a Cell, most notably
* LocalActorRef and RepointableActorRef. The former specializes the return
* type of `underlying` so that follow-up calls can use invokevirtual instead
* of invokeinterface.
*/
private[akka] abstract class ActorRefWithCell extends InternalActorRef { this: ActorRefScope
def underlying: Cell
}
/**
* This is an internal look-up failure token, not useful for anything else.
*/
@ -227,31 +251,24 @@ private[akka] class LocalActorRef private[akka] (
_system: ActorSystemImpl,
_props: Props,
_supervisor: InternalActorRef,
override val path: ActorPath,
val systemService: Boolean = false,
_receiveTimeout: Option[Duration] = None)
extends InternalActorRef with LocalRef {
override val path: ActorPath)
extends ActorRefWithCell with LocalRef {
/*
* actorCell.start() publishes actorCell & this to the dispatcher, which
* means that messages may be processed theoretically before the constructor
* ends. The JMM guarantees visibility for final fields only after the end
* of the constructor, so publish the actorCell safely by making it a
* @volatile var which is NOT TO BE WRITTEN TO. The alternative would be to
* move start() outside of the constructor, which would basically require
* us to use purely factory methods for creating LocalActorRefs.
* Safe publication of this classs fields is guaranteed by mailbox.setActor()
* which is called indirectly from actorCell.start() (if youre wondering why
* this is at all important, remember that under the JMM final fields are only
* frozen at the _end_ of the constructor, but we are publishing this before
* that is reached).
*/
@volatile
private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout)
private val actorCell: ActorCell = newActorCell(_system, this, _props, _supervisor)
actorCell.start()
protected def newActorCell(
system: ActorSystemImpl,
ref: InternalActorRef,
props: Props,
supervisor: InternalActorRef,
receiveTimeout: Option[Duration]): ActorCell =
new ActorCell(system, ref, props, supervisor, receiveTimeout)
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
_supervisor.sendSystemMessage(akka.dispatch.Supervise(this))
protected def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell =
new ActorCell(system, ref, props, supervisor)
protected def actorContext: ActorContext = actorCell
@ -320,9 +337,9 @@ private[akka] class LocalActorRef private[akka] (
// ========= AKKA PROTECTED FUNCTIONS =========
protected[akka] def underlying: ActorCell = actorCell
def underlying: ActorCell = actorCell
override def sendSystemMessage(message: SystemMessage): Unit = underlying.dispatcher.systemDispatch(underlying, message)
override def sendSystemMessage(message: SystemMessage): Unit = actorCell.sendSystemMessage(message)
override def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender)
@ -409,16 +426,26 @@ private[akka] object DeadLetterActorRef {
*
* INTERNAL API
*/
private[akka] class EmptyLocalActorRef(
override val provider: ActorRefProvider,
private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
override val path: ActorPath,
val eventStream: EventStream) extends MinimalActorRef {
override def isTerminated(): Boolean = true
override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message)
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
case d: DeadLetter // do NOT form endless loops, since deadLetters will resend!
case _ eventStream.publish(DeadLetter(message, sender, this))
case d: DeadLetter specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend!
case _ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
}
protected def specialHandle(msg: Any): Boolean = msg match {
case w: Watch
if (w.watchee == this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false)
true
case _: Unwatch true // Just ignore
case _ false
}
}
@ -428,12 +455,22 @@ private[akka] class EmptyLocalActorRef(
*
* INTERNAL API
*/
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream)
extends EmptyLocalActorRef(_provider, _path, _eventStream) {
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider,
_path: ActorPath,
_eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) {
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
case d: DeadLetter eventStream.publish(d)
case _ eventStream.publish(DeadLetter(message, sender, this))
case d: DeadLetter if (!specialHandle(d.message)) eventStream.publish(d)
case _ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
}
override protected def specialHandle(msg: Any): Boolean = msg match {
case w: Watch
if (w.watchee != this && w.watcher != this)
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false)
true
case w: Unwatch true // Just ignore
case _ false
}
@throws(classOf[java.io.ObjectStreamException])

View file

@ -8,8 +8,8 @@ import java.util.concurrent.atomic.AtomicLong
import akka.dispatch._
import akka.routing._
import akka.AkkaException
import akka.util.{ Switch, Helpers }
import akka.event._
import akka.util.{ NonFatal, Switch, Helpers }
/**
* Interface for all ActorRef providers to implement.
@ -26,23 +26,18 @@ trait ActorRefProvider {
/**
* Reference to the supervisor used for all top-level user actors.
*/
def guardian: InternalActorRef
def guardian: LocalActorRef
/**
* Reference to the supervisor used for all top-level system actors.
*/
def systemGuardian: InternalActorRef
def systemGuardian: LocalActorRef
/**
* Dead letter destination for this provider.
*/
def deadLetters: ActorRef
/**
* Reference to the death watch service.
*/
def deathWatch: DeathWatch
/**
* The root path for all actors within this actor system, including remote
* address if enabled.
@ -109,7 +104,8 @@ trait ActorRefProvider {
path: ActorPath,
systemService: Boolean,
deploy: Option[Deploy],
lookupDeploy: Boolean): InternalActorRef
lookupDeploy: Boolean,
async: Boolean): InternalActorRef
/**
* Create actor reference for a specified local or remote path. If no such
@ -162,8 +158,9 @@ trait ActorRefFactory {
* INTERNAL USE ONLY
*/
protected def provider: ActorRefProvider
/**
* Returns the default MessageDispatcher used by this ActorRefFactory
* Returns the default MessageDispatcher associated with this ActorRefFactory
*/
implicit def dispatcher: MessageDispatcher
@ -339,8 +336,6 @@ class LocalActorRefProvider(
override val deadLetters: InternalActorRef = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
override val deathWatch: DeathWatch = new LocalDeathWatch(1024) //TODO make configrable
/*
* generate name for temporary actor refs
*/
@ -379,8 +374,8 @@ class LocalActorRefProvider(
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
message match {
case Supervise(child) // TODO register child in some map to keep track of it and enable shutdown after all dead
case ChildTerminated(child) stop()
case Supervise(_) // TODO register child in some map to keep track of it and enable shutdown after all dead
case ChildTerminated(_) stop()
case _ log.error(this + " received unexpected system message [" + message + "]")
}
}
@ -409,8 +404,8 @@ class LocalActorRefProvider(
def receive = {
case Terminated(_) context.stop(self)
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case NonFatal(e) Status.Failure(e) })
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case NonFatal(e) Status.Failure(e) })
case StopChild(child) context.stop(child); sender ! "ok"
case m deadLetters ! DeadLetter(m, sender, self)
}
@ -441,8 +436,8 @@ class LocalActorRefProvider(
def receive = {
case Terminated(_) eventStream.stopDefaultLoggers(); context.stop(self)
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case e: Exception e }) // FIXME shouldn't this use NonFatal & Status.Failure?
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case NonFatal(e) Status.Failure(e) })
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case NonFatal(e) Status.Failure(e) })
case StopChild(child) context.stop(child); sender ! "ok"
case m deadLetters ! DeadLetter(m, sender, self)
}
@ -479,27 +474,18 @@ class LocalActorRefProvider(
private val guardianProps = Props(new Guardian)
lazy val rootGuardian: InternalActorRef =
new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) {
object Extra {
def unapply(s: String): Option[InternalActorRef] = extraNames.get(s)
}
new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath) {
override def getParent: InternalActorRef = this
override def getSingleChild(name: String): InternalActorRef = {
name match {
override def getSingleChild(name: String): InternalActorRef = name match {
case "temp" tempContainer
case Extra(e) e
case _ super.getSingleChild(name)
}
case other extraNames.get(other).getOrElse(super.getSingleChild(other))
}
}
lazy val guardian: InternalActorRef =
actorOf(system, guardianProps, rootGuardian, rootPath / "user", true, None, false)
lazy val guardian: LocalActorRef = new LocalActorRef(system, guardianProps, rootGuardian, rootPath / "user")
lazy val systemGuardian: InternalActorRef =
actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system", true, None, false)
lazy val systemGuardian: LocalActorRef =
new LocalActorRef(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system")
lazy val tempContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log)
@ -516,8 +502,8 @@ class LocalActorRefProvider(
def init(_system: ActorSystemImpl) {
system = _system
// chain death watchers so that killing guardian stops the application
deathWatch.subscribe(systemGuardian, guardian)
deathWatch.subscribe(rootGuardian, systemGuardian)
systemGuardian.sendSystemMessage(Watch(guardian, systemGuardian))
rootGuardian.sendSystemMessage(Watch(systemGuardian, rootGuardian))
eventStream.startDefaultLoggers(_system)
}
@ -553,32 +539,20 @@ class LocalActorRefProvider(
}
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean): InternalActorRef = {
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
props.routerConfig match {
case NoRouter new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
case NoRouter
if (async) new RepointableActorRef(system, props, supervisor, path).initialize()
else new LocalActorRef(system, props, supervisor, path)
case router
val lookup = if (lookupDeploy) deployer.lookup(path) else None
val fromProps = Iterator(props.deploy.copy(routerConfig = props.deploy.routerConfig withFallback router))
val d = fromProps ++ deploy.iterator ++ lookup.iterator reduce ((a, b) b withFallback a)
new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path)
val ref = new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path).initialize()
if (async) ref else ref.activate()
}
}
def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None
}
class LocalDeathWatch(val mapSize: Int) extends DeathWatch with ActorClassification {
override def publish(event: Event): Unit = {
val monitors = dissociate(classify(event))
if (monitors.nonEmpty) monitors.foreach(_ ! event)
}
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = {
if (!super.subscribe(subscriber, to)) {
subscriber ! Terminated(to)
false
} else true
}
}

View file

@ -13,7 +13,6 @@ import java.io.Closeable
import akka.dispatch.Await.{ Awaitable, CanAwait }
import akka.util._
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
import collection.immutable.Stack
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
import java.util.concurrent.TimeUnit.MILLISECONDS
@ -305,8 +304,9 @@ abstract class ActorSystem extends ActorRefFactory {
implicit def dispatcher: MessageDispatcher
/**
* Register a block of code (callback) to run after all actors in this actor system have
* been stopped. Multiple code blocks may be registered by calling this method multiple times.
* Register a block of code (callback) to run after ActorSystem.shutdown has been issued and
* all actors in this actor system have been stopped.
* Multiple code blocks may be registered by calling this method multiple times.
* The callbacks will be run sequentially in reverse order of registration, i.e.
* last registration is run first.
*
@ -317,8 +317,9 @@ abstract class ActorSystem extends ActorRefFactory {
def registerOnTermination[T](code: T): Unit
/**
* Register a block of code (callback) to run after all actors in this actor system have
* been stopped. Multiple code blocks may be registered by calling this method multiple times.
* Register a block of code (callback) to run after ActorSystem.shutdown has been issued and
* all actors in this actor system have been stopped.
* Multiple code blocks may be registered by calling this method multiple times.
* The callbacks will be run sequentially in reverse order of registration, i.e.
* last registration is run first.
*
@ -408,11 +409,6 @@ abstract class ExtendedActorSystem extends ActorSystem {
*/
def systemGuardian: InternalActorRef
/**
* Implementation of the mechanism which is used for watch()/unwatch().
*/
def deathWatch: DeathWatch
/**
* A ThreadFactory that can be used if the transport needs to create any Threads
*/
@ -426,6 +422,13 @@ abstract class ExtendedActorSystem extends ActorSystem {
* creation.
*/
def dynamicAccess: DynamicAccess
/**
* For debugging: traverse actor hierarchy and make string representation.
* Careful, this may OOM on large actor systems, and it is only meant for
* helping debugging in case something already went terminally wrong.
*/
private[akka] def printTree: String
}
private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem {
@ -433,7 +436,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
if (!name.matches("""^[a-zA-Z0-9][a-zA-Z0-9-]*$"""))
throw new IllegalArgumentException(
"invalid ActorSystem name [" + name +
"], must contain only word characters (i.e. [a-zA-Z_0-9] plus non-leading '-')")
"], must contain only word characters (i.e. [a-zA-Z0-9] plus non-leading '-')")
import ActorSystem._
@ -483,29 +486,11 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
protected def systemImpl: ActorSystemImpl = this
private[akka] def systemActorOf(props: Props, name: String): ActorRef = {
implicit val timeout = settings.CreationTimeout
Await.result(systemGuardian ? CreateChild(props, name), timeout.duration) match {
case ref: ActorRef ref
case ex: Exception throw ex
}
}
private[akka] def systemActorOf(props: Props, name: String): ActorRef = systemGuardian.underlying.attachChild(props, name)
def actorOf(props: Props, name: String): ActorRef = {
implicit val timeout = settings.CreationTimeout
Await.result(guardian ? CreateChild(props, name), timeout.duration) match {
case ref: ActorRef ref
case ex: Exception throw ex
}
}
def actorOf(props: Props, name: String): ActorRef = guardian.underlying.attachChild(props, name)
def actorOf(props: Props): ActorRef = {
implicit val timeout = settings.CreationTimeout
Await.result(guardian ? CreateRandomNameChild(props), timeout.duration) match {
case ref: ActorRef ref
case ex: Exception throw ex
}
}
def actorOf(props: Props): ActorRef = guardian.underlying.attachChild(props)
def stop(actor: ActorRef): Unit = {
implicit val timeout = settings.CreationTimeout
@ -547,17 +532,19 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
//FIXME Why do we need this at all?
val deadLetterQueue: MessageQueue = new MessageQueue {
def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
def enqueue(receiver: ActorRef, envelope: Envelope): Unit =
deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver)
def dequeue() = null
def hasMessages = false
def numberOfMessages = 0
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = ()
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = ()
}
//FIXME Why do we need this at all?
val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) {
val deadLetterMailbox: Mailbox = new Mailbox(deadLetterQueue) {
becomeClosed()
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = deadLetters ! DeadLetter(handle, receiver, receiver)
def systemDrain(): SystemMessage = null
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit =
deadLetters ! DeadLetter(handle, receiver, receiver)
def systemDrain(newContents: SystemMessage): SystemMessage = null
def hasSystemMessages = false
}
@ -568,9 +555,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
def terminationFuture: Future[Unit] = provider.terminationFuture
def lookupRoot: InternalActorRef = provider.rootGuardian
def guardian: InternalActorRef = provider.guardian
def systemGuardian: InternalActorRef = provider.systemGuardian
def deathWatch: DeathWatch = provider.deathWatch
def guardian: LocalActorRef = provider.guardian
def systemGuardian: LocalActorRef = provider.systemGuardian
def /(actorName: String): ActorPath = guardian.path / actorName
def /(path: Iterable[String]): ActorPath = guardian.path / path
@ -694,10 +680,35 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
override def toString: String = lookupRoot.path.root.address.toString
override def printTree: String = {
def printNode(node: ActorRef, indent: String): String = {
node match {
case wc: ActorRefWithCell
val cell = wc.underlying
indent + "-> " + node.path.name + " " + Logging.simpleName(node) + " " +
(cell match {
case real: ActorCell if (real.actor ne null) real.actor.getClass else "null"
case _ Logging.simpleName(cell)
}) +
" " + (cell.childrenRefs match {
case ActorCell.TerminatingChildrenContainer(_, toDie, reason)
"Terminating(" + reason + ")" +
(toDie.toSeq.sorted mkString ("\n" + indent + " toDie: ", "\n" + indent + " ", ""))
case x Logging.simpleName(x)
}) +
(if (cell.childrenRefs.children.isEmpty) "" else "\n") +
(cell.childrenRefs.children.toSeq.sorted map (printNode(_, indent + " |")) mkString ("\n"))
case _
indent + node.path.name + " " + Logging.simpleName(node)
}
}
printNode(actorFor("/"), "")
}
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
private val lock = new ReentrantGuard
private var callbacks: Stack[Runnable] = _ //non-volatile since guarded by the lock
lock withGuard { callbacks = Stack.empty[Runnable] }
private var callbacks: List[Runnable] = _ //non-volatile since guarded by the lock
lock withGuard { callbacks = Nil }
private val latch = new CountDownLatch(1)
@ -706,17 +717,17 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
case 0 throw new RejectedExecutionException("Must be called prior to system shutdown.")
case _ lock withGuard {
if (latch.getCount == 0) throw new RejectedExecutionException("Must be called prior to system shutdown.")
else callbacks = callbacks.push(callback)
else callbacks ::= callback
}
}
}
final def run(): Unit = lock withGuard {
@tailrec def runNext(c: Stack[Runnable]): Stack[Runnable] = c.headOption match {
case None Stack.empty[Runnable]
case Some(callback)
try callback.run() catch { case e log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) }
runNext(c.pop)
@tailrec def runNext(c: List[Runnable]): List[Runnable] = c match {
case Nil Nil
case callback :: rest
try callback.run() catch { case NonFatal(e) log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) }
runNext(rest)
}
try { callbacks = runNext(callbacks) } finally latch.countDown()
}

View file

@ -9,11 +9,22 @@ import scala.collection.JavaConversions._
import java.lang.{ Iterable JIterable }
import akka.util.Duration
/**
* INTERNAL API
*/
private[akka] sealed trait ChildStats
/**
* INTERNAL API
*/
private[akka] case object ChildNameReserved extends ChildStats
/**
* ChildRestartStats is the statistics kept by every parent Actor for every child Actor
* and is used for SupervisorStrategies to know how to deal with problems that occur for the children.
*/
case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) {
case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L)
extends ChildStats {
//FIXME How about making ChildRestartStats immutable and then move these methods into the actual supervisor strategies?
def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean =

View file

@ -6,7 +6,6 @@ package akka.actor
import akka.dispatch._
import akka.japi.Creator
import collection.immutable.Stack
import akka.routing._
/**
@ -186,5 +185,10 @@ case class Props(
* able to optimize serialization.
*/
private[akka] case class FromClassCreator(clazz: Class[_ <: Actor]) extends Function0[Actor] {
def apply(): Actor = clazz.newInstance
def apply(): Actor = try clazz.newInstance catch {
case iae: IllegalAccessException
val ctor = clazz.getDeclaredConstructor()
ctor.setAccessible(true)
ctor.newInstance()
}
}

View file

@ -0,0 +1,214 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import akka.util.Unsafe
import scala.annotation.tailrec
import akka.dispatch.SystemMessage
import akka.dispatch.Mailbox
import akka.dispatch.Terminate
import akka.dispatch.Envelope
import akka.dispatch.Supervise
import akka.dispatch.Create
import akka.dispatch.MessageDispatcher
import java.util.concurrent.locks.ReentrantLock
import akka.event.Logging.Warning
import scala.collection.mutable.Queue
/**
* This actor ref starts out with some dummy cell (by default just enqueuing
* messages into vectors protected by ReentrantLock), it must be initialize()d
* before it can be sent to, and it will be activate()d by its supervisor in
* response to the Supervise() message, which will replace the contained Cell
* with a fully functional one, transfer all messages from dummy to real queue
* and swap out the cell ref.
*/
private[akka] class RepointableActorRef(
val system: ActorSystemImpl,
val props: Props,
val supervisor: InternalActorRef,
val path: ActorPath)
extends ActorRefWithCell with RepointableRef {
import AbstractActorRef.cellOffset
@volatile private var _cellDoNotCallMeDirectly: Cell = _
def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell]
@tailrec final def swapCell(next: Cell): Cell = {
val old = underlying
if (Unsafe.instance.compareAndSwapObject(this, cellOffset, old, next)) old else swapCell(next)
}
/**
* Initialize: make a dummy cell which holds just a mailbox, then tell our
* supervisor that we exist so that he can create the real Cell in
* handleSupervise().
*
* Call twice on your own peril!
*
* This is protected so that others can have different initialization.
*/
def initialize(): this.type = {
swapCell(new UnstartedCell(system, this, props, supervisor))
supervisor.sendSystemMessage(Supervise(this))
this
}
/**
* This method is supposed to be called by the supervisor in handleSupervise()
* to replace the UnstartedCell with the real one. It assumes no concurrent
* modification of the `underlying` field, though it is safe to send messages
* at any time.
*/
def activate(): this.type = {
underlying match {
case u: UnstartedCell u.replaceWith(newCell())
case _ // this happens routinely for things which were created async=false
}
this
}
/**
* This is called by activate() to obtain the cell which is to replace the
* unstarted cell. The cell must be fully functional.
*/
def newCell(): Cell = new ActorCell(system, this, props, supervisor).start()
def suspend(): Unit = underlying.suspend()
def resume(): Unit = underlying.resume()
def stop(): Unit = underlying.stop()
def restart(cause: Throwable): Unit = underlying.restart(cause)
def isStarted: Boolean = !underlying.isInstanceOf[UnstartedCell]
def isTerminated: Boolean = underlying.isTerminated
def provider: ActorRefProvider = system.provider
def isLocal: Boolean = underlying.isLocal
def getParent: InternalActorRef = underlying.parent
def getChild(name: Iterator[String]): InternalActorRef =
if (name.hasNext) {
name.next match {
case ".." getParent.getChild(name)
case "" getChild(name)
case other
underlying.childrenRefs.getByName(other) match {
case Some(crs) crs.child.asInstanceOf[InternalActorRef].getChild(name)
case None Nobody
}
}
} else this
def !(message: Any)(implicit sender: ActorRef = null) = underlying.tell(message, sender)
def sendSystemMessage(message: SystemMessage) = underlying.sendSystemMessage(message)
@throws(classOf[java.io.ObjectStreamException])
protected def writeReplace(): AnyRef = SerializedActorRef(path)
}
private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: RepointableActorRef, val props: Props, val supervisor: InternalActorRef)
extends Cell {
/*
* This lock protects all accesses to this cells queues. It also ensures
* safe switching to the started ActorCell.
*/
val lock = new ReentrantLock
// use Envelope to keep on-send checks in the same place
val queue: Queue[Envelope] = Queue()
val systemQueue: Queue[SystemMessage] = Queue()
def replaceWith(cell: Cell): Unit = {
lock.lock()
try {
/*
* The CallingThreadDispatcher nicely dives under the ReentrantLock and
* breaks things by enqueueing into stale queues from within the message
* processing which happens in-line for sendSystemMessage() and tell().
* Since this is the only possible way to f*ck things up within this
* lock, double-tap (well, N-tap, really); concurrent modification is
* still not possible because were the only thread accessing the queues.
*/
var interrupted = false
while (systemQueue.nonEmpty || queue.nonEmpty) {
while (systemQueue.nonEmpty) {
val msg = systemQueue.dequeue()
try cell.sendSystemMessage(msg)
catch {
case _: InterruptedException interrupted = true
}
}
if (queue.nonEmpty) {
val envelope = queue.dequeue()
try cell.tell(envelope.message, envelope.sender)
catch {
case _: InterruptedException interrupted = true
}
}
}
if (interrupted) throw new InterruptedException
} finally try
self.swapCell(cell)
finally
lock.unlock()
}
def system: ActorSystem = systemImpl
def suspend(): Unit = {}
def resume(): Unit = {}
def restart(cause: Throwable): Unit = {}
def stop(): Unit = sendSystemMessage(Terminate())
def isTerminated: Boolean = false
def parent: InternalActorRef = supervisor
def childrenRefs: ActorCell.ChildrenContainer = ActorCell.EmptyChildrenContainer
def tell(message: Any, sender: ActorRef): Unit = {
lock.lock()
try {
if (self.underlying eq this) queue enqueue Envelope(message, sender, system)
else self.underlying.tell(message, sender)
} finally {
lock.unlock()
}
}
def sendSystemMessage(msg: SystemMessage): Unit = {
lock.lock()
try {
if (self.underlying eq this) systemQueue enqueue msg
else self.underlying.sendSystemMessage(msg)
} finally {
lock.unlock()
}
}
def isLocal = true
def hasMessages: Boolean = {
lock.lock()
try {
if (self.underlying eq this) !queue.isEmpty
else self.underlying.hasMessages
} finally {
lock.unlock()
}
}
def numberOfMessages: Int = {
lock.lock()
try {
if (self.underlying eq this) queue.size
else self.underlying.numberOfMessages
} finally {
lock.unlock()
}
}
}

View file

@ -592,7 +592,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
/**
* Returns the akka.actor.Props representation of this TypedProps
*/
def actorProps(): Props = if (dispatcher == Props().dispatcher) Props() else Props(dispatcher = dispatcher)
def actorProps(): Props = if (dispatcher == Props.default.dispatcher) Props.default else Props(dispatcher = dispatcher)
}
/**

View file

@ -16,8 +16,10 @@ import akka.event.Logging.LogEventException
import akka.jsr166y.{ ForkJoinTask, ForkJoinPool }
import akka.util.{ Unsafe, Duration, NonFatal, Index }
final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) {
if (message.isInstanceOf[AnyRef]) {
final case class Envelope private (val message: Any, val sender: ActorRef)
object Envelope {
def apply(message: Any, sender: ActorRef, system: ActorSystem): Envelope = {
val msg = message.asInstanceOf[AnyRef]
if (msg eq null) throw new InvalidMessageException("Message is null")
if (system.settings.SerializeAllMessages && !msg.isInstanceOf[NoSerializationVerificationNeeded]) {
@ -30,6 +32,7 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS
}
}
}
new Envelope(message, sender)
}
}
@ -102,11 +105,15 @@ private[akka] case class ChildTerminated(child: ActorRef) extends SystemMessage
/**
* INTERNAL API
*/
private[akka] case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch
private[akka] case class Watch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to establish a DeathWatch
/**
* INTERNAL API
*/
private[akka] case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch
private[akka] case class Unwatch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to tear down a DeathWatch
/**
* INTERNAL API
*/
private[akka] case object NoMessage extends SystemMessage // switched into the mailbox to signal termination
final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () Unit) extends Runnable {
def run(): Unit =
@ -224,7 +231,7 @@ private[akka] object MessageDispatcher {
} {
val status = if (a.isTerminated) " (terminated)" else " (alive)"
val messages = a match {
case l: LocalActorRef " " + l.underlying.mailbox.numberOfMessages + " messages"
case r: ActorRefWithCell " " + r.underlying.numberOfMessages + " messages"
case _ " " + a.getClass
}
val parent = a match {
@ -261,7 +268,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
/**
* Creates and returns a mailbox for the given actor.
*/
protected[akka] def createMailbox(actor: ActorCell): Mailbox //FIXME should this really be private[akka]?
protected[akka] def createMailbox(actor: Cell): Mailbox //FIXME should this really be private[akka]?
/**
* Identifier of this dispatcher, corresponds to the full key
@ -310,16 +317,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
case 0
shutdownSchedule match {
case UNSCHEDULED
if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) {
scheduleShutdownAction()
()
} else ifSensibleToDoSoThenScheduleShutdown()
if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) scheduleShutdownAction()
else ifSensibleToDoSoThenScheduleShutdown()
case SCHEDULED
if (updateShutdownSchedule(SCHEDULED, RESCHEDULED)) ()
else ifSensibleToDoSoThenScheduleShutdown()
case RESCHEDULED ()
case RESCHEDULED
}
case _ ()
case _
}
private def scheduleShutdownAction(): Unit = {
@ -349,9 +354,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
protected[akka] def unregister(actor: ActorCell) {
if (debug) actors.remove(this, actor.self)
addInhabitants(-1)
val mailBox = actor.mailbox
val mailBox = actor.swapMailbox(deadLetterMailbox)
mailBox.becomeClosed() // FIXME reschedule in tell if possible race with cleanUp is detected in order to properly clean up
actor.mailbox = deadLetterMailbox
mailBox.cleanUp()
}
@ -359,7 +363,6 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
@tailrec
final def run() {
shutdownSchedule match {
case UNSCHEDULED ()
case SCHEDULED
try {
if (inhabitants == 0) shutdown() //Warning, racy
@ -369,6 +372,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
case RESCHEDULED
if (updateShutdownSchedule(RESCHEDULED, SCHEDULED)) scheduleShutdownAction()
else run()
case UNSCHEDULED
}
}
}

View file

@ -9,6 +9,7 @@ import annotation.tailrec
import akka.util.{ Duration, Helpers }
import java.util.{ Comparator, Iterator }
import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
import akka.actor.ActorSystemImpl
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -46,26 +47,25 @@ class BalancingDispatcher(
/**
* INTERNAL USE ONLY
*/
private[akka] val messageQueue: MessageQueue = mailboxType.create(None)
private[akka] val messageQueue: MessageQueue = mailboxType.create(None, None)
private class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue {
private class SharingMailbox(val system: ActorSystemImpl, _messageQueue: MessageQueue)
extends Mailbox(_messageQueue) with DefaultSystemMessageQueue {
override def cleanUp(): Unit = {
val dlq = system.deadLetterMailbox
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
if (hasSystemMessages) {
val dlq = actor.systemImpl.deadLetterMailbox
var message = systemDrain()
var message = systemDrain(NoMessage)
while (message ne null) {
// message must be virgin before being able to systemEnqueue again
val next = message.next
message.next = null
dlq.systemEnqueue(actor.self, message)
dlq.systemEnqueue(system.deadLetters, message)
message = next
}
}
}
}
protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor, messageQueue)
protected[akka] override def createMailbox(actor: akka.actor.Cell): Mailbox = new SharingMailbox(actor.systemImpl, messageQueue)
protected[akka] override def register(actor: ActorCell): Unit = {
super.register(actor)

View file

@ -82,7 +82,8 @@ class Dispatcher(
/**
* INTERNAL USE ONLY
*/
protected[akka] def createMailbox(actor: ActorCell): Mailbox = new Mailbox(actor, mailboxType.create(Some(actor))) with DefaultSystemMessageQueue
protected[akka] def createMailbox(actor: akka.actor.Cell): Mailbox =
new Mailbox(mailboxType.create(Some(actor.self), Some(actor.system))) with DefaultSystemMessageQueue
/**
* INTERNAL USE ONLY

View file

@ -6,18 +6,12 @@ package akka.dispatch
import akka.AkkaException
import java.util.{ Comparator, PriorityQueue, Queue, Deque }
import akka.util._
import akka.actor.{ ActorCell, ActorRef }
import akka.actor.{ ActorCell, ActorRef, Cell }
import java.util.concurrent._
import annotation.tailrec
import akka.event.Logging.Error
import akka.actor.ActorContext
import com.typesafe.config.Config
import akka.actor.ActorSystem
/**
* This exception normally is thrown when a bounded mailbox is over capacity
*/
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
import akka.actor._
/**
* INTERNAL API
@ -48,11 +42,32 @@ private[akka] object Mailbox {
*
* INTERNAL API
*/
private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: MessageQueue)
private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
extends SystemMessageQueue with Runnable {
import Mailbox._
/*
* This is needed for actually executing the mailbox, i.e. invoking the
* ActorCell. There are situations (e.g. RepointableActorRef) where a Mailbox
* is constructed but we know that we will not execute it, in which case this
* will be null. It must be a var to support switching into an active
* mailbox, should the owning ActorRef turn local.
*
* ANOTHER THING, IMPORTANT:
*
* actorCell.start() publishes actorCell & self to the dispatcher, which
* means that messages may be processed theoretically before selfs constructor
* ends. The JMM guarantees visibility for final fields only after the end
* of the constructor, so safe publication requires that THIS WRITE BELOW
* stay as it is.
*/
@volatile
var actor: ActorCell = _
def setActor(cell: ActorCell): Unit = actor = cell
def dispatcher: MessageDispatcher = actor.dispatcher
/**
* Try to enqueue the message to this queue, or throw an exception.
*/
@ -169,6 +184,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
*/
protected final def systemQueueGet: SystemMessage =
Unsafe.instance.getObjectVolatile(this, AbstractMailbox.systemMessageOffset).asInstanceOf[SystemMessage]
protected final def systemQueuePut(_old: SystemMessage, _new: SystemMessage): Boolean =
Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old, _new)
@ -207,25 +223,49 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
}
}
/**
* Will at least try to process all queued system messages: in case of
* failure simply drop and go on to the next, because there is nothing to
* restart here (failure is in ActorCell somewhere ). In case the mailbox
* becomes closed (because of processing a Terminate message), dump all
* already dequeued message to deadLetters.
*/
final def processAllSystemMessages() {
var nextMessage = systemDrain()
try {
var failure: Throwable = null
var nextMessage = systemDrain(null)
while ((nextMessage ne null) && !isClosed) {
if (debug) println(actor.self + " processing system message " + nextMessage + " with " + actor.childrenRefs)
actor systemInvoke nextMessage
val msg = nextMessage
nextMessage = nextMessage.next
// dont ever execute normal message when system message present!
if (nextMessage eq null) nextMessage = systemDrain()
}
msg.next = null
if (debug) println(actor.self + " processing system message " + msg + " with " + actor.childrenRefs)
try {
actor systemInvoke msg
} catch {
case NonFatal(e)
actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
throw e
if (failure eq null) failure = e
actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system message " + msg + ": " + e.getMessage))
}
// dont ever execute normal message when system message present!
if ((nextMessage eq null) && !isClosed) nextMessage = systemDrain(null)
}
/*
* if we closed the mailbox, we must dump the remaining system messages
* to deadLetters (this is essential for DeathWatch)
*/
val dlm = actor.systemImpl.deadLetterMailbox
while (nextMessage ne null) {
val msg = nextMessage
nextMessage = nextMessage.next
msg.next = null
try dlm.systemEnqueue(actor.self, msg)
catch {
case NonFatal(e) actor.system.eventStream.publish(
Error(e, actor.self.path.toString, this.getClass, "error while enqueuing " + msg + " to deadLetters: " + e.getMessage))
}
}
@inline
final def dispatcher: MessageDispatcher = actor.dispatcher
// if something happened while processing, fail this actor (most probable: exception in supervisorStrategy)
if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage)
}
/**
* Overridable callback to clean up the mailbox,
@ -235,8 +275,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
protected[dispatch] def cleanUp(): Unit =
if (actor ne null) { // actor is null for the deadLetterMailbox
val dlm = actor.systemImpl.deadLetterMailbox
if (hasSystemMessages) {
var message = systemDrain()
var message = systemDrain(NoMessage)
while (message ne null) {
// message must be virgin before being able to systemEnqueue again
val next = message.next
@ -244,10 +283,9 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
dlm.systemEnqueue(actor.self, message)
message = next
}
}
if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run()
messageQueue.cleanUp(actor, actor.systemImpl.deadLetterQueue)
messageQueue.cleanUp(actor.self, actor.systemImpl.deadLetterQueue)
}
}
@ -285,7 +323,7 @@ trait MessageQueue {
* which is passed in. The owner of this MessageQueue is passed in if
* available (e.g. for creating DeadLetters()), /deadletters otherwise.
*/
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit
}
/**
@ -300,7 +338,7 @@ private[akka] trait SystemMessageQueue {
/**
* Dequeue all messages from system queue and return them as single-linked list.
*/
def systemDrain(): SystemMessage
def systemDrain(newContents: SystemMessage): SystemMessage
def hasSystemMessages: Boolean
}
@ -313,8 +351,11 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
@tailrec
final def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit = {
assert(message.next eq null)
if (Mailbox.debug) println(actor.self + " having enqueued " + message)
if (Mailbox.debug) println(receiver + " having enqueued " + message)
val head = systemQueueGet
if (head == NoMessage) {
if (actor ne null) actor.systemImpl.deadLetterMailbox.systemEnqueue(receiver, message)
} else {
/*
* this write is safely published by the compareAndSet contained within
* systemQueuePut; Intra-Thread Semantics on page 12 of the JSR133 spec
@ -327,14 +368,16 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
systemEnqueue(receiver, message)
}
}
}
@tailrec
final def systemDrain(): SystemMessage = {
final def systemDrain(newContents: SystemMessage): SystemMessage = {
val head = systemQueueGet
if (systemQueuePut(head, null)) SystemMessage.reverse(head) else systemDrain()
if (systemQueuePut(head, newContents)) SystemMessage.reverse(head) else systemDrain(newContents)
}
def hasSystemMessages: Boolean = systemQueueGet ne null
}
/**
@ -344,11 +387,11 @@ trait QueueBasedMessageQueue extends MessageQueue {
def queue: Queue[Envelope]
def numberOfMessages = queue.size
def hasMessages = !queue.isEmpty
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = {
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
if (hasMessages) {
var envelope = dequeue
while (envelope ne null) {
deadLetters.enqueue(owner.self, envelope)
deadLetters.enqueue(owner, envelope)
envelope = dequeue
}
}
@ -372,13 +415,11 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
def pushTimeOut: Duration
override def queue: BlockingQueue[Envelope]
def enqueue(receiver: ActorRef, handle: Envelope) {
def enqueue(receiver: ActorRef, handle: Envelope): Unit =
if (pushTimeOut.length > 0) {
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
}
if (!queue.offer(handle, pushTimeOut.length, pushTimeOut.unit))
receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver)
} else queue put handle
}
def dequeue(): Envelope = queue.poll()
}
@ -410,27 +451,35 @@ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
override def queue: BlockingDeque[Envelope]
def enqueue(receiver: ActorRef, handle: Envelope): Unit =
if (pushTimeOut.length > 0)
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
}
else queue put handle
if (pushTimeOut.length > 0) {
if (!queue.offer(handle, pushTimeOut.length, pushTimeOut.unit))
receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver)
} else queue put handle
def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit =
if (pushTimeOut.length > 0)
queue.offerFirst(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
}
else queue putFirst handle
if (pushTimeOut.length > 0) {
if (!queue.offerFirst(handle, pushTimeOut.length, pushTimeOut.unit))
receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver)
} else queue putFirst handle
def dequeue(): Envelope = queue.poll()
}
/**
* MailboxType is a factory to create MessageQueues for an optionally provided ActorContext
* MailboxType is a factory to create MessageQueues for an optionally
* provided ActorContext.
*
* <b>Possibly Important Notice</b>
*
* When implementing a custom mailbox type, be aware that there is special
* semantics attached to `system.actorOf()` in that sending to the returned
* ActorRef mayfor a short period of timeenqueue the messages first in a
* dummy queue. Top-level actors are created in two steps, and only after the
* guardian actor has performed that second step will all previously sent
* messages be transferred from the dummy queue into the real mailbox.
*/
trait MailboxType {
def create(owner: Option[ActorContext]): MessageQueue
def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue
}
/**
@ -440,7 +489,7 @@ case class UnboundedMailbox() extends MailboxType {
def this(settings: ActorSystem.Settings, config: Config) = this()
final override def create(owner: Option[ActorContext]): MessageQueue =
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
final def queue: Queue[Envelope] = this
}
@ -457,7 +506,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
final override def create(owner: Option[ActorContext]): MessageQueue =
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new LinkedBlockingQueue[Envelope](capacity) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
final def queue: BlockingQueue[Envelope] = this
final val pushTimeOut = BoundedMailbox.this.pushTimeOut
@ -470,7 +519,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat
*/
class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope], final val initialCapacity: Int) extends MailboxType {
def this(cmp: Comparator[Envelope]) = this(cmp, 11)
final override def create(owner: Option[ActorContext]): MessageQueue =
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new PriorityBlockingQueue[Envelope](initialCapacity, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
final def queue: Queue[Envelope] = this
}
@ -485,7 +534,7 @@ class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val cap
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
final override def create(owner: Option[ActorContext]): MessageQueue =
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
final def queue: BlockingQueue[Envelope] = this
final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut
@ -499,7 +548,7 @@ case class UnboundedDequeBasedMailbox() extends MailboxType {
def this(settings: ActorSystem.Settings, config: Config) = this()
final override def create(owner: Option[ActorContext]): MessageQueue =
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new LinkedBlockingDeque[Envelope]() with DequeBasedMessageQueue with UnboundedDequeBasedMessageQueueSemantics {
final val queue = this
}
@ -516,7 +565,7 @@ case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTime
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative")
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedDequeBasedMailbox can not be null")
final override def create(owner: Option[ActorContext]): MessageQueue =
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new LinkedBlockingDeque[Envelope](capacity) with DequeBasedMessageQueue with BoundedDequeBasedMessageQueueSemantics {
final val queue = this
final val pushTimeOut = BoundedDequeBasedMailbox.this.pushTimeOut

View file

@ -29,30 +29,20 @@ object ThreadPoolConfig {
val defaultTimeout: Duration = Duration(60000L, TimeUnit.MILLISECONDS)
val defaultRejectionPolicy: RejectedExecutionHandler = new SaneRejectedExecutionHandler()
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int = {
import scala.math.{ min, max }
min(max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
}
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int =
math.min(math.max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory =
() new ArrayBlockingQueue[Runnable](capacity, fair)
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = () new ArrayBlockingQueue[Runnable](capacity, fair)
def synchronousQueue(fair: Boolean): QueueFactory =
() new SynchronousQueue[Runnable](fair)
def synchronousQueue(fair: Boolean): QueueFactory = () new SynchronousQueue[Runnable](fair)
def linkedBlockingQueue(): QueueFactory =
() new LinkedBlockingQueue[Runnable]()
def linkedBlockingQueue(): QueueFactory = () new LinkedBlockingQueue[Runnable]()
def linkedBlockingQueue(capacity: Int): QueueFactory =
() new LinkedBlockingQueue[Runnable](capacity)
def linkedBlockingQueue(capacity: Int): QueueFactory = () new LinkedBlockingQueue[Runnable](capacity)
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory =
() queue
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = () queue
def reusableQueue(queueFactory: QueueFactory): QueueFactory = {
val queue = queueFactory()
() queue
}
def reusableQueue(queueFactory: QueueFactory): QueueFactory = reusableQueue(queueFactory())
}
/**
@ -157,7 +147,8 @@ case class ThreadPoolConfigBuilder(config: ThreadPoolConfig) {
def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = newQueueFactory))
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder = fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder =
fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
}
object MonitorableThreadFactory {

View file

@ -1,19 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.event
import akka.actor._
/**
* 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 LocalDeathWatch for semantics
*/
abstract class DeathWatch extends ActorEventBus with ActorClassifier {
type Event = Terminated
protected final def classify(event: Event): Classifier = event.actor
}

View file

@ -324,7 +324,17 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
case some some 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)
def subscribe(subscriber: Subscriber, to: Classifier): Boolean =
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
else if (to eq null) throw new IllegalArgumentException("Classifier is null")
else associate(to, subscriber)
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean =
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
else if (from eq null) throw new IllegalArgumentException("Classifier is null")
else dissociate(from, subscriber)
def unsubscribe(subscriber: Subscriber): Unit =
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
else dissociate(subscriber)
}

View file

@ -39,17 +39,20 @@ class EventStream(private val debug: Boolean = false) extends LoggingBus with Su
}
override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
if (subscriber eq null) throw new IllegalArgumentException("subscriber is null")
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "subscribing " + subscriber + " to channel " + channel))
super.subscribe(subscriber, channel)
}
override def unsubscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
if (subscriber eq null) throw new IllegalArgumentException("subscriber is null")
val ret = super.unsubscribe(subscriber, channel)
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from channel " + channel))
ret
}
override def unsubscribe(subscriber: ActorRef) {
if (subscriber eq null) throw new IllegalArgumentException("subscriber is null")
super.unsubscribe(subscriber)
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from all channels"))
}

View file

@ -875,3 +875,16 @@ class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class
protected def notifyInfo(message: String): Unit = bus.publish(Info(logSource, logClass, message))
protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message))
}
private[akka] object NoLogging extends LoggingAdapter {
def isErrorEnabled = false
def isWarningEnabled = false
def isInfoEnabled = false
def isDebugEnabled = false
protected def notifyError(message: String): Unit = ()
protected def notifyError(cause: Throwable, message: String): Unit = ()
protected def notifyWarning(message: String): Unit = ()
protected def notifyInfo(message: String): Unit = ()
protected def notifyDebug(message: String): Unit = ()
}

View file

@ -4,12 +4,10 @@
package akka.pattern
import java.util.concurrent.TimeoutException
import akka.dispatch.{ Promise, Terminate, SystemMessage, Future }
import akka.event.DeathWatch
import akka.util.Timeout
import annotation.tailrec
import akka.util.Unsafe
import akka.actor._
import akka.dispatch._
import akka.util.{ NonFatal, Timeout, Unsafe }
/**
* This is what is used to complete a Future that is returned from an ask/? call,
@ -164,6 +162,7 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
extends MinimalActorRef {
import PromiseActorRef._
import AbstractPromiseActorRef.stateOffset
import AbstractPromiseActorRef.watchedByOffset
/**
* As an optimization for the common (local) case we only register this PromiseActorRef
@ -180,14 +179,43 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
@volatile
private[this] var _stateDoNotCallMeDirectly: AnyRef = _
@inline
private def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
@volatile
private[this] var _watchedByDoNotCallMeDirectly: Set[ActorRef] = ActorCell.emptyActorRefSet
@inline
private def updateState(oldState: AnyRef, newState: AnyRef): Boolean = Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
private[this] def watchedBy: Set[ActorRef] = Unsafe.instance.getObjectVolatile(this, watchedByOffset).asInstanceOf[Set[ActorRef]]
@inline
private def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
private[this] def updateWatchedBy(oldWatchedBy: Set[ActorRef], newWatchedBy: Set[ActorRef]): Boolean =
Unsafe.instance.compareAndSwapObject(this, watchedByOffset, oldWatchedBy, newWatchedBy)
@tailrec // Returns false if the Promise is already completed
private[this] final def addWatcher(watcher: ActorRef): Boolean = watchedBy match {
case null false
case other updateWatchedBy(other, other + watcher) || addWatcher(watcher)
}
@tailrec
private[this] final def remWatcher(watcher: ActorRef): Unit = watchedBy match {
case null ()
case other if (!updateWatchedBy(other, other - watcher)) remWatcher(watcher)
}
@tailrec
private[this] final def clearWatchers(): Set[ActorRef] = watchedBy match {
case null ActorCell.emptyActorRefSet
case other if (!updateWatchedBy(other, null)) clearWatchers() else other
}
@inline
private[this] def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
@inline
private[this] def updateState(oldState: AnyRef, newState: AnyRef): Boolean =
Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
@inline
private[this] def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
override def getParent: InternalActorRef = provider.tempContainer
@ -218,19 +246,24 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
override def !(message: Any)(implicit sender: ActorRef = null): Unit = state match {
case Stopped | _: StoppedWithPath provider.deadLetters ! message
case _
val completedJustNow = result.tryComplete {
case _ if (!(result.tryComplete {
message match {
case Status.Success(r) Right(r)
case Status.Failure(f) Left(f)
case other Right(other)
}
}
if (!completedJustNow) provider.deadLetters ! message
})) provider.deadLetters ! message
}
override def sendSystemMessage(message: SystemMessage): Unit = message match {
case _: Terminate stop()
case Watch(watchee, watcher)
if (watchee == this && watcher != this) {
if (!addWatcher(watcher)) watcher ! Terminated(watchee)(existenceConfirmed = true)
} else System.err.println("BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, this))
case Unwatch(watchee, watcher)
if (watchee == this && watcher != this) remWatcher(watcher)
else System.err.println("BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, this))
case _
}
@ -241,23 +274,20 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
@tailrec
override def stop(): Unit = {
def ensurePromiseCompleted(): Unit =
def ensureCompleted(): Unit = {
if (!result.isCompleted) result.tryComplete(Left(new ActorKilledException("Stopped")))
state match {
case null
// if path was never queried nobody can possibly be watching us, so we don't have to publish termination either
if (updateState(null, Stopped)) ensurePromiseCompleted()
else stop()
case p: ActorPath
if (updateState(p, StoppedWithPath(p))) {
try {
ensurePromiseCompleted()
provider.deathWatch.publish(Terminated(this))
} finally {
provider.unregisterTempActor(p)
val watchers = clearWatchers()
if (!watchers.isEmpty) {
val termination = Terminated(this)(existenceConfirmed = true)
watchers foreach { w try w.tell(termination, this) catch { case NonFatal(t) /* FIXME LOG THIS */ } }
}
} else stop()
case Stopped | _: StoppedWithPath
}
state match {
case null // if path was never queried nobody can possibly be watching us, so we don't have to publish termination either
if (updateState(null, Stopped)) ensureCompleted() else stop()
case p: ActorPath
if (updateState(p, StoppedWithPath(p))) { try ensureCompleted() finally provider.unregisterTempActor(p) } else stop()
case Stopped | _: StoppedWithPath // already stopped
case Registering stop() // spin until registration is completed before stopping
}
}

View file

@ -0,0 +1,560 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.pattern
import java.util.concurrent.atomic.{ AtomicInteger, AtomicLong, AtomicBoolean }
import akka.AkkaException
import akka.actor.Scheduler
import akka.dispatch.{ Future, ExecutionContext, Await, Promise }
import akka.util.{ Deadline, Duration, NonFatal, Unsafe }
import akka.util.duration._
import util.control.NoStackTrace
import java.util.concurrent.{ Callable, CopyOnWriteArrayList }
/**
* Companion object providing factory methods for Circuit Breaker which runs callbacks in caller's thread
*/
object CircuitBreaker {
/**
* Synchronous execution context to run in caller's thread - used by companion object factory methods
*/
private[CircuitBreaker] val syncExecutionContext = new ExecutionContext {
def execute(runnable: Runnable): Unit = runnable.run()
def reportFailure(t: Throwable): Unit = ()
}
/**
* Callbacks run in caller's thread when using withSyncCircuitBreaker, and in same ExecutionContext as the passed
* in Future when using withCircuitBreaker. To use another ExecutionContext for the callbacks you can specify the
* executor in the constructor.
*
* @param scheduler Reference to Akka scheduler
* @param maxFailures Maximum number of failures before opening the circuit
* @param callTimeout [[akka.util.Duration]] of time after which to consider a call a failure
* @param resetTimeout [[akka.util.Duration]] of time after which to attempt to close the circuit
*/
def apply(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration): CircuitBreaker =
new CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)(syncExecutionContext)
/**
* Java API alias for apply
*
* @param scheduler Reference to Akka scheduler
* @param maxFailures Maximum number of failures before opening the circuit
* @param callTimeout [[akka.util.Duration]] of time after which to consider a call a failure
* @param resetTimeout [[akka.util.Duration]] of time after which to attempt to close the circuit
*/
def create(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration): CircuitBreaker =
apply(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)
}
/**
* Provides circuit breaker functionality to provide stability when working with "dangerous" operations, e.g. calls to
* remote systems
*
* Transitions through three states:
* - In *Closed* state, calls pass through until the `maxFailures` count is reached. This causes the circuit breaker
* to open. Both exceptions and calls exceeding `callTimeout` are considered failures.
* - In *Open* state, calls fail-fast with an exception. After `resetTimeout`, circuit breaker transitions to
* half-open state.
* - In *Half-Open* state, the first call will be allowed through, if it succeeds the circuit breaker will reset to
* closed state. If it fails, the circuit breaker will re-open to open state. All calls beyond the first that
* execute while the first is running will fail-fast with an exception.
*
*
* @param scheduler Reference to Akka scheduler
* @param maxFailures Maximum number of failures before opening the circuit
* @param callTimeout [[akka.util.Duration]] of time after which to consider a call a failure
* @param resetTimeout [[akka.util.Duration]] of time after which to attempt to close the circuit
* @param executor [[akka.dispatch.ExecutionContext]] used for execution of state transition listeners
*/
class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)(implicit executor: ExecutionContext) extends AbstractCircuitBreaker {
def this(executor: ExecutionContext, scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration) = {
this(scheduler, maxFailures, callTimeout, resetTimeout)(executor)
}
/**
* Holds reference to current state of CircuitBreaker - *access only via helper methods*
*/
@volatile
private[this] var _currentStateDoNotCallMeDirectly: State = Closed
/**
* Helper method for access to underlying state via Unsafe
*
* @param oldState Previous state on transition
* @param newState Next state on transition
* @return Whether the previous state matched correctly
*/
@inline
private[this] def swapState(oldState: State, newState: State): Boolean =
Unsafe.instance.compareAndSwapObject(this, AbstractCircuitBreaker.stateOffset, oldState, newState)
/**
* Helper method for accessing underlying state via Unsafe
*
* @return Reference to current state
*/
@inline
private[this] def currentState: State =
Unsafe.instance.getObjectVolatile(this, AbstractCircuitBreaker.stateOffset).asInstanceOf[State]
/**
* Wraps invocations of asynchronous calls that need to be protected
*
* @param body Call needing protected
* @tparam T return type from call
* @return [[akka.dispatch.Future]] containing the call result
*/
def withCircuitBreaker[T](body: Future[T]): Future[T] = {
currentState.invoke(body)
}
/**
* Java API for withCircuitBreaker
*
* @param body Call needing protected
* @tparam T return type from call
* @return [[akka.dispatch.Future]] containing the call result
*/
def callWithCircuitBreaker[T](body: Callable[Future[T]]): Future[T] = {
withCircuitBreaker(body.call)
}
/**
* Wraps invocations of synchronous calls that need to be protected
*
* Calls are run in caller's thread
*
* @param body Call needing protected
* @tparam T return type from call
* @return The result of the call
*/
def withSyncCircuitBreaker[T](body: T): T = {
Await.result(withCircuitBreaker(
{
try
Promise.successful(body)(CircuitBreaker.syncExecutionContext)
catch {
case NonFatal(t) Promise.failed(t)(CircuitBreaker.syncExecutionContext)
}
}), callTimeout)
}
/**
* Java API for withSyncCircuitBreaker
*
* @param body Call needing protected
* @tparam T return type from call
* @return The result of the call
*/
def callWithSyncCircuitBreaker[T](body: Callable[T]): T = {
withSyncCircuitBreaker(body.call)
}
/**
* Adds a callback to execute when circuit breaker opens
*
* The callback is run in the [[akka.dispatch.ExecutionContext]] supplied in the constructor.
*
* @param callback Handler to be invoked on state change
* @tparam T Type supplied to assist with type inference, otherwise ignored by implementation
* @return CircuitBreaker for fluent usage
*/
def onOpen[T](callback: T): CircuitBreaker = {
Open.addListener(() callback)
this
}
/**
* Java API for onOpen
*
* @param callback Handler to be invoked on state change
* @tparam T Type supplied to assist with type inference, otherwise ignored by implementation
* @return CircuitBreaker for fluent usage
*/
def onOpen[T](callback: Callable[T]): CircuitBreaker = {
onOpen(callback.call)
}
/**
* Adds a callback to execute when circuit breaker transitions to half-open
*
* The callback is run in the [[akka.dispatch.ExecutionContext]] supplied in the constructor.
*
* @param callback Handler to be invoked on state change
* @tparam T Type supplied to assist with type inference, otherwise ignored by implementation
* @return CircuitBreaker for fluent usage
*/
def onHalfOpen[T](callback: T): CircuitBreaker = {
HalfOpen.addListener(() callback)
this
}
/**
* JavaAPI for onHalfOpen
*
* @param callback Handler to be invoked on state change
* @tparam T Type supplied to assist with type inference, otherwise ignored by implementation
* @return CircuitBreaker for fluent usage
*/
def onHalfOpen[T](callback: Callable[T]): CircuitBreaker = {
onHalfOpen(callback.call)
}
/**
* Adds a callback to execute when circuit breaker state closes
*
* The callback is run in the [[akka.dispatch.ExecutionContext]] supplied in the constructor.
*
* @param callback Handler to be invoked on state change
* @tparam T Type supplied to assist with type inference, otherwise ignored by implementation
* @return CircuitBreaker for fluent usage
*/
def onClose[T](callback: T): CircuitBreaker = {
Closed.addListener(() callback)
this
}
/**
* JavaAPI for onClose
*
* @param callback Handler to be invoked on state change
* @tparam T Type supplied to assist with type inference, otherwise ignored by implementation
* @return CircuitBreaker for fluent usage
*/
def onClose[T](callback: Callable[T]): CircuitBreaker = {
onClose(callback.call)
}
/**
* Retrieves current failure count.
*
* @return count
*/
private[akka] def currentFailureCount: Int = Closed.get
/**
* Implements consistent transition between states
*
* @param fromState State being transitioning from
* @param toState State being transitioning from
* @throws IllegalStateException if an invalid transition is attempted
*/
private def transition(fromState: State, toState: State): Unit = {
if (swapState(fromState, toState))
toState.enter()
else
throw new IllegalStateException("Illegal transition attempted from: " + fromState + " to " + toState)
}
/**
* Trips breaker to an open state. This is valid from Closed or Half-Open states.
*
* @param fromState State we're coming from (Closed or Half-Open)
*/
private def tripBreaker(fromState: State): Unit = {
transition(fromState, Open)
}
/**
* Resets breaker to a closed state. This is valid from an Half-Open state only.
*
*/
private def resetBreaker(): Unit = {
transition(HalfOpen, Closed)
}
/**
* Attempts to reset breaker by transitioning to a half-open state. This is valid from an Open state only.
*
*/
private def attemptReset(): Unit = {
transition(Open, HalfOpen)
}
/**
* Internal state abstraction
*/
private sealed trait State {
private val listeners = new CopyOnWriteArrayList[() _]
/**
* Add a listener function which is invoked on state entry
*
* @param listener listener implementation
* @tparam T return type of listener, not used - but supplied for type inference purposes
*/
def addListener[T](listener: () T) {
listeners add listener
}
/**
* Test for whether listeners exist
*
* @return whether listeners exist
*/
private def hasListeners: Boolean = !listeners.isEmpty
/**
* Notifies the listeners of the transition event via a Future executed in implicit parameter ExecutionContext
*
* @return Promise which executes listener in supplied [[akka.dispatch.ExecutionContext]]
*/
protected def notifyTransitionListeners() {
if (hasListeners) {
val iterator = listeners.iterator
while (iterator.hasNext) {
val listener = iterator.next
//FIXME per @viktorklang: it's a bit wasteful to create Futures for one-offs, just use EC.execute instead
Future(listener())
}
}
}
/**
* Shared implementation of call across all states. Thrown exception or execution of the call beyond the allowed
* call timeout is counted as a failed call, otherwise a successful call
*
* @param body Implementation of the call
* @tparam T Return type of the call's implementation
* @return Future containing the result of the call
*/
def callThrough[T](body: Future[T]): Future[T] = {
val deadline = callTimeout.fromNow
val bodyFuture = try body catch {
case NonFatal(t) Promise.failed(t)
}
bodyFuture onFailure {
case _ callFails()
} onSuccess {
case _
if (deadline.isOverdue()) callFails()
else callSucceeds()
}
}
/**
* Abstract entry point for all states
*
* @param body Implementation of the call that needs protected
* @tparam T Return type of protected call
* @return Future containing result of protected call
*/
def invoke[T](body: Future[T]): Future[T]
/**
* Invoked when call succeeds
*
*/
def callSucceeds(): Unit
/**
* Invoked when call fails
*
*/
def callFails(): Unit
/**
* Invoked on the transitioned-to state during transition. Notifies listeners after invoking subclass template
* method _enter
*
*/
final def enter(): Unit = {
_enter()
notifyTransitionListeners()
}
/**
* Template method for concrete traits
*
*/
def _enter(): Unit
}
/**
* Concrete implementation of Closed state
*/
private object Closed extends AtomicInteger with State {
/**
* Implementation of invoke, which simply attempts the call
*
* @param body Implementation of the call that needs protected
* @tparam T Return type of protected call
* @return Future containing result of protected call
*/
override def invoke[T](body: Future[T]): Future[T] = {
callThrough(body)
}
/**
* On successful call, the failure count is reset to 0
*
* @return
*/
override def callSucceeds(): Unit = { set(0) }
/**
* On failed call, the failure count is incremented. The count is checked against the configured maxFailures, and
* the breaker is tripped if we have reached maxFailures.
*
* @return
*/
override def callFails(): Unit = {
if (incrementAndGet() == maxFailures) tripBreaker(Closed)
}
/**
* On entry of this state, failure count is reset.
*
* @return
*/
override def _enter(): Unit = {
set(0)
}
/**
* Override for more descriptive toString
*
* @return
*/
override def toString: String = {
"Closed with failure count = " + get()
}
}
/**
* Concrete implementation of half-open state
*/
private object HalfOpen extends AtomicBoolean(true) with State {
/**
* Allows a single call through, during which all other callers fail-fast. If the call fails, the breaker reopens.
* If the call succeeds the breaker closes.
*
* @param body Implementation of the call that needs protected
* @tparam T Return type of protected call
* @return Future containing result of protected call
*/
override def invoke[T](body: Future[T]): Future[T] = {
if (compareAndSet(true, false))
callThrough(body)
else
Promise.failed[T](new CircuitBreakerOpenException(Duration.Zero))
}
/**
* Reset breaker on successful call.
*
* @return
*/
override def callSucceeds(): Unit = { resetBreaker() }
/**
* Reopen breaker on failed call.
*
* @return
*/
override def callFails(): Unit = { tripBreaker(HalfOpen) }
/**
* On entry, guard should be reset for that first call to get in
*
* @return
*/
override def _enter(): Unit = {
set(true)
}
/**
* Override for more descriptive toString
*
* @return
*/
override def toString: String = {
"Half-Open currently testing call for success = " + get()
}
}
/**
* Concrete implementation of Open state
*/
private object Open extends AtomicLong with State {
/**
* Fail-fast on any invocation
*
* @param body Implementation of the call that needs protected
* @tparam T Return type of protected call
* @return Future containing result of protected call
*/
override def invoke[T](body: Future[T]): Future[T] = {
Promise.failed[T](new CircuitBreakerOpenException(remainingTimeout().timeLeft))
}
/**
* Calculate remaining timeout to inform the caller in case a backoff algorithm is useful
*
* @return [[akka.util.Deadline]] to when the breaker will attempt a reset by transitioning to half-open
*/
private def remainingTimeout(): Deadline = get match {
case 0L Deadline.now
case t (t.millis + resetTimeout).fromNow
}
/**
* No-op for open, calls are never executed so cannot succeed or fail
*
* @return
*/
override def callSucceeds(): Unit = {}
/**
* No-op for open, calls are never executed so cannot succeed or fail
*
* @return
*/
override def callFails(): Unit = {}
/**
* On entering this state, schedule an attempted reset via [[akka.actor.Scheduler]] and store the entry time to
* calculate remaining time before attempted reset.
*
* @return
*/
override def _enter(): Unit = {
set(System.currentTimeMillis)
scheduler.scheduleOnce(resetTimeout) {
attemptReset()
}
}
/**
* Override for more descriptive toString
*
* @return
*/
override def toString: String = {
"Open"
}
}
}
/**
* Exception thrown when Circuit Breaker is open.
*
* @param remainingDuration Stores remaining time before attempting a reset. Zero duration means the breaker is
* currently in half-open state.
* @param message Defaults to "Circuit Breaker is open; calls are failing fast"
*/
class CircuitBreakerOpenException(
val remainingDuration: Duration,
message: String = "Circuit Breaker is open; calls are failing fast")
extends AkkaException(message) with NoStackTrace

View file

@ -4,9 +4,9 @@
package akka.pattern
import akka.dispatch.{ Promise, Future }
import akka.actor._
import akka.util.{ Timeout, Duration }
import akka.dispatch.{ Unwatch, Watch, Promise, Future }
trait GracefulStopSupport {
/**
@ -34,18 +34,21 @@ trait GracefulStopSupport {
* is completed with failure [[akka.pattern.AskTimeoutException]].
*/
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = {
if (target.isTerminated) {
Promise.successful(true)
} else system match {
if (target.isTerminated) Promise.successful(true)
else system match {
case e: ExtendedActorSystem
val internalTarget = target.asInstanceOf[InternalActorRef]
val ref = PromiseActorRef(e.provider, Timeout(timeout))
e.deathWatch.subscribe(ref, target)
ref.result onComplete {
case Right(Terminated(`target`)) () // Ignore
case _ e.deathWatch.unsubscribe(ref, target)
} // Just making sure we're not leaking here
internalTarget.sendSystemMessage(Watch(target, ref))
ref.result onComplete { // Just making sure we're not leaking here
case Right(Terminated(`target`)) ()
case _ internalTarget.sendSystemMessage(Unwatch(target, ref))
}
target ! PoisonPill
ref.result map { case Terminated(`target`) true }
ref.result map {
case Terminated(`target`) true
case _ false
}
case s throw new IllegalArgumentException("Unknown ActorSystem implementation: '" + s + "'")
}
}

View file

@ -23,42 +23,28 @@ import scala.runtime.ScalaRunTime
* send a message to on (or more) of these actors.
*/
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
extends LocalActorRef(
_system,
_props.copy(creator = () _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
_supervisor,
_path) {
extends RepointableActorRef(_system, _props, _supervisor, _path) {
// verify that a BalancingDispatcher is not used with a Router
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher))
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) {
throw new ConfigurationException(
"Configuration for actor [" + _path.toString +
"] is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
/*
* CAUTION: RoutedActorRef is PROBLEMATIC
* ======================================
*
* We are constructing/assembling the children outside of the scope of the
* Router actor, inserting them in its childrenRef list, which is not at all
* synchronized. This is done exactly once at start-up, all other accesses
* are done from the Router actor. This means that the only thing which is
* really hairy is making sure that the Router does not touch its childrenRefs
* before we are done with them: lock the monitor of the actor cell (hence the
* override of newActorCell) and use that to block the Router constructor for
* as long as it takes to setup the RoutedActorRef itself.
*/
override def newActorCell(
system: ActorSystemImpl,
ref: InternalActorRef,
props: Props,
supervisor: InternalActorRef,
receiveTimeout: Option[Duration]): ActorCell = {
val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout)
Unsafe.instance.monitorEnter(cell)
cell
"Configuration for " + this +
" is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
}
_props.routerConfig.verifyConfig()
override def newCell(): Cell = new RoutedActorCell(system, this, props, supervisor)
}
private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef)
extends ActorCell(
_system,
_ref,
_props.copy(creator = () _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
_supervisor) {
private[akka] val routerConfig = _props.routerConfig
private[akka] val routeeProps = _props.copy(routerConfig = NoRouter)
private[akka] val resizeInProgress = new AtomicBoolean
@ -72,9 +58,8 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
private var _routeeProvider: RouteeProvider = _
def routeeProvider = _routeeProvider
val route =
try {
_routeeProvider = routerConfig.createRouteeProvider(actorContext)
val route = {
_routeeProvider = routerConfig.createRouteeProvider(this)
val r = routerConfig.createRoute(routeeProps, routeeProvider)
// initial resize, before message send
routerConfig.resizer foreach { r
@ -82,21 +67,19 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
r.resize(routeeProps, routeeProvider)
}
r
} finally {
assert(Thread.holdsLock(actorContext))
Unsafe.instance.monitorExit(actorContext) // unblock Routers constructor
}
if (routerConfig.resizer.isEmpty && _routees.isEmpty)
throw new ActorInitializationException("router " + routerConfig + " did not register routees!")
start()
/*
* end of construction
*/
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
case _: AutoReceivedMessage Destination(this, this) :: Nil
case Terminated(_) Destination(this, this) :: Nil
case _: AutoReceivedMessage Destination(self, self) :: Nil
case CurrentRoutees
sender ! RouterRoutees(_routees)
Nil
@ -114,7 +97,7 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = {
_routees = _routees ++ newRoutees
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
newRoutees foreach underlying.watch
newRoutees foreach watch
}
/**
@ -125,13 +108,13 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
*/
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = {
_routees = _routees diff abandonedRoutees
abandonedRoutees foreach underlying.unwatch
abandonedRoutees foreach unwatch
}
override def !(message: Any)(implicit sender: ActorRef = null): Unit = {
override def tell(message: Any, sender: ActorRef): Unit = {
resize()
val s = if (sender eq null) underlying.system.deadLetters else sender
val s = if (sender eq null) system.deadLetters else sender
val msg = message match {
case Broadcast(m) m
@ -139,15 +122,18 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
}
applyRoute(s, message) match {
case Destination(_, x) :: Nil if x eq this super.!(message)(s)
case refs refs foreach (p p.recipient.!(msg)(p.sender))
case Destination(_, x) :: Nil if x == self super.tell(message, s)
case refs
refs foreach (p
if (p.recipient == self) super.tell(msg, p.sender)
else p.recipient.!(msg)(p.sender))
}
}
def resize(): Unit = {
for (r routerConfig.resizer) {
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true))
super.!(Router.Resize)
super.tell(Router.Resize, self)
}
}
}
@ -204,6 +190,11 @@ trait RouterConfig {
*/
def resizer: Option[Resizer] = None
/**
* Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early.
*/
def verifyConfig(): Unit = {}
}
/**
@ -219,7 +210,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
* Not thread safe, but intended to be called from protected points, such as
* `RouterConfig.createRoute` and `Resizer.resize`.
*/
def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = routedRef.addRoutees(routees)
def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = routedCell.addRoutees(routees)
/**
* Adds the routees to the router.
@ -239,7 +230,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
* Not thread safe, but intended to be called from protected points, such as
* `Resizer.resize`.
*/
def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = routedRef.removeRoutees(routees)
def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = routedCell.removeRoutees(routees)
def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] =
(nrOfInstances, routees) match {
@ -256,9 +247,9 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
/**
* All routees of the router
*/
def routees: IndexedSeq[ActorRef] = routedRef.routees
def routees: IndexedSeq[ActorRef] = routedCell.routees
private def routedRef = context.self.asInstanceOf[RoutedActorRef]
private def routedCell = context.asInstanceOf[RoutedActorCell]
}
/**
@ -290,12 +281,9 @@ trait CustomRoute {
*/
trait Router extends Actor {
// make sure that we synchronize properly to get the childrenRefs into our CPU cache
val ref = context.synchronized {
self match {
case x: RoutedActorRef x
case _ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
}
val ref = context match {
case x: RoutedActorCell x
case _ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass)
}
final def receive = ({
@ -409,8 +397,10 @@ class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
def this() = this(Dispatchers.DefaultDispatcherId)
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
override def verifyConfig(): Unit =
throw new ConfigurationException("router needs external configuration from file (e.g. application.conf)")
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy
}
@ -766,9 +756,11 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
* routers based on mailbox and actor internal state.
*/
protected def isProcessingMessage(a: ActorRef): Boolean = a match {
case x: LocalActorRef
val cell = x.underlying
cell.mailbox.isScheduled && cell.currentMessage != null
case x: ActorRefWithCell
x.underlying match {
case cell: ActorCell cell.mailbox.isScheduled && cell.currentMessage != null
case _ false
}
case _ false
}
@ -780,7 +772,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
* routers based on mailbox and actor internal state.
*/
protected def hasMessages(a: ActorRef): Boolean = a match {
case x: LocalActorRef x.underlying.mailbox.hasMessages
case x: ActorRefWithCell x.underlying.hasMessages
case _ false
}
@ -791,7 +783,11 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
* routers based on mailbox and actor internal state.
*/
protected def isSuspended(a: ActorRef): Boolean = a match {
case x: LocalActorRef x.underlying.mailbox.isSuspended
case x: ActorRefWithCell
x.underlying match {
case cell: ActorCell cell.mailbox.isSuspended
case _ true
}
case _ false
}
@ -802,7 +798,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
* routers based on mailbox and actor internal state.
*/
protected def numberOfMessages(a: ActorRef): Int = a match {
case x: LocalActorRef x.underlying.mailbox.numberOfMessages
case x: ActorRefWithCell x.underlying.numberOfMessages
case _ 0
}
@ -1275,13 +1271,21 @@ case class DefaultResizer(
*/
def pressure(routees: IndexedSeq[ActorRef]): Int = {
routees count {
case a: LocalActorRef
val cell = a.underlying
case a: ActorRefWithCell
a.underlying match {
case cell: ActorCell
pressureThreshold match {
case 1 cell.mailbox.isScheduled && cell.mailbox.hasMessages
case i if i < 1 cell.mailbox.isScheduled && cell.currentMessage != null
case threshold cell.mailbox.numberOfMessages >= threshold
}
case cell
pressureThreshold match {
case 1 cell.hasMessages
case i if i < 1 true // unstarted cells are always busy, for example
case threshold cell.numberOfMessages >= threshold
}
}
case x
false
}

View file

@ -9,16 +9,22 @@ import TimeUnit._
import java.lang.{ Double JDouble }
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class Deadline private (time: Duration) {
case class Deadline private (time: Duration) extends Ordered[Deadline] {
def +(other: Duration): Deadline = copy(time = time + other)
def -(other: Duration): Deadline = copy(time = time - other)
def -(other: Deadline): Duration = time - other.time
def timeLeft: Duration = this - Deadline.now
def hasTimeLeft(): Boolean = !isOverdue() //Code reuse FTW
def isOverdue(): Boolean = (time.toNanos - System.nanoTime()) < 0
def compare(that: Deadline) = this.time compare that.time
}
object Deadline {
def now: Deadline = Deadline(Duration(System.nanoTime, NANOSECONDS))
implicit object DeadlineIsOrdered extends Ordering[Deadline] {
def compare(a: Deadline, b: Deadline) = a compare b
}
}
object Duration {

View file

@ -97,7 +97,7 @@ object Agent {
*/
class Agent[T](initialValue: T, system: ActorSystem) {
private val ref = Ref(initialValue)
private val updater = system.actorOf(Props(new AgentUpdater(this, ref))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow?
private val updater = system.actorOf(Props(new AgentUpdater(this, ref))).asInstanceOf[InternalActorRef] //TODO can we avoid this somehow?
/**
* Read the internal state of the agent.

View file

@ -25,15 +25,21 @@ akka {
periodic-tasks-initial-delay = 1s
# how often should the node send out gossip information?
gossip-frequency = 1s
gossip-interval = 1s
# how often should the node send out heartbeats?
heartbeat-interval = 1s
# how often should the leader perform maintenance tasks?
leader-actions-frequency = 1s
leader-actions-interval = 1s
# how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring?
unreachable-nodes-reaper-frequency = 1s
unreachable-nodes-reaper-interval = 1s
# A joining node stops sending heartbeats to the node to join if it hasn't become member
# of the cluster within this deadline.
join-timeout = 60s
# accrual failure detection config
failure-detector {
# defines the failure detector threshold
@ -41,9 +47,34 @@ akka {
# 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
threshold = 8
threshold = 8.0
# Minimum standard deviation to use for the normal distribution in
# AccrualFailureDetector. Too low standard deviation might result in
# too much sensitivity for sudden, but normal, deviations in heartbeat
# inter arrival times.
min-std-deviation = 100 ms
# Number of potentially lost/delayed heartbeats that will be
# accepted before considering it to be an anomaly.
# It is a factor of heartbeat-interval.
# This margin is important to be able to survive sudden, occasional,
# pauses in heartbeat arrivals, due to for example garbage collect or
# network drop.
acceptable-heartbeat-pause = 3s
implementation-class = "akka.cluster.AccrualFailureDetector"
max-sample-size = 1000
}
# If the tick-duration of the default scheduler is longer than the tick-duration
# configured here a dedicated scheduler will be used for periodic tasks of the cluster,
# otherwise the default scheduler is used.
# See akka.scheduler settings for more details about the HashedWheelTimer.
scheduler {
tick-duration = 33ms
ticks-per-wheel = 512
}
}
}

View file

@ -4,39 +4,102 @@
package akka.cluster
import akka.actor.{ ActorSystem, Address }
import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
import akka.remote.RemoteActorRefProvider
import akka.event.Logging
import scala.collection.immutable.Map
import scala.annotation.tailrec
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.TimeUnit.NANOSECONDS
import akka.util.Duration
import akka.util.duration._
object AccrualFailureDetector {
private def realClock: () Long = () NANOSECONDS.toMillis(System.nanoTime)
}
/**
* 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
*
* The suspicion level of failure is given by a value called φ (phi).
* The basic idea of the φ failure detector is to express the value of φ on a scale that
* is dynamically adjusted to reflect current network conditions. A configurable
* threshold is used to decide if φ is considered to be a failure.
*
* The value of φ is calculated as:
*
* {{{
* φ = -log10(1 - F(timeSinceLastHeartbeat)
* }}}
* where F is the cumulative distribution function of a normal distribution with mean
* and standard deviation estimated from historical heartbeat inter-arrival times.
*
*
* @param system Belongs to the [[akka.actor.ActorSystem]]. Used for logging.
*
* @param threshold 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/>
* Default threshold is 8, but can be configured in the Akka config.
*
* @param maxSampleSize Number of samples to use for calculation of mean and standard deviation of
* inter-arrival times.
*
* @param minStdDeviation Minimum standard deviation to use for the normal distribution used when calculating phi.
* Too low standard deviation might result in too much sensitivity for sudden, but normal, deviations
* in heartbeat inter arrival times.
*
* @param acceptableHeartbeatPause Duration corresponding to number of potentially lost/delayed
* heartbeats that will be accepted before considering it to be an anomaly.
* This margin is important to be able to survive sudden, occasional, pauses in heartbeat
* arrivals, due to for example garbage collect or network drop.
*
* @param firstHeartbeatEstimate Bootstrap the stats with heartbeats that corresponds to
* to this duration, with a with rather high standard deviation (since environment is unknown
* in the beginning)
*
* @clock The clock, returning current time in milliseconds, but can be faked for testing
* purposes. It is only used for measuring intervals (duration).
*
*/
class AccrualFailureDetector(
system: ActorSystem,
address: Address,
val threshold: Int = 8,
val maxSampleSize: Int = 1000,
val timeMachine: () Long = System.currentTimeMillis) {
val system: ActorSystem,
val threshold: Double,
val maxSampleSize: Int,
val minStdDeviation: Duration,
val acceptableHeartbeatPause: Duration,
val firstHeartbeatEstimate: Duration,
val clock: () Long) extends FailureDetector {
private final val PhiFactor = 1.0 / math.log(10.0)
import AccrualFailureDetector._
/**
* Constructor that picks configuration from the settings.
*/
def this(
system: ActorSystem,
settings: ClusterSettings,
clock: () Long = AccrualFailureDetector.realClock) =
this(
system,
settings.FailureDetectorThreshold,
settings.FailureDetectorMaxSampleSize,
settings.FailureDetectorAcceptableHeartbeatPause,
settings.FailureDetectorMinStdDeviation,
settings.HeartbeatInterval,
clock)
private val log = Logging(system, "FailureDetector")
/**
* Holds the failure statistics for a specific node Address.
*/
private case class FailureStats(mean: Double = 0.0D, variance: Double = 0.0D, deviation: Double = 0.0D)
// guess statistics for first heartbeat,
// important so that connections with only one heartbeat becomes unavailable
private val firstHeartbeat: HeartbeatHistory = {
// bootstrap with 2 entries with rather high standard deviation
val mean = firstHeartbeatEstimate.toMillis
val stdDeviation = mean / 4
HeartbeatHistory(maxSampleSize) :+ (mean - stdDeviation) :+ (mean + stdDeviation)
}
private val acceptableHeartbeatPauseMillis = acceptableHeartbeatPause.toMillis
/**
* Implement using optimistic lockless concurrency, all state is represented
@ -44,8 +107,7 @@ class AccrualFailureDetector(
*/
private case class State(
version: Long = 0L,
failureStats: Map[Address, FailureStats] = Map.empty[Address, FailureStats],
intervalHistory: Map[Address, IndexedSeq[Long]] = Map.empty[Address, IndexedSeq[Long]],
history: Map[Address, HeartbeatHistory] = Map.empty,
timestamps: Map[Address, Long] = Map.empty[Address, Long],
explicitRemovals: Set[Address] = Set.empty[Address])
@ -62,118 +124,91 @@ class AccrualFailureDetector(
*/
@tailrec
final def heartbeat(connection: Address) {
log.debug("Node [{}] - Heartbeat from connection [{}] ", address, connection)
log.debug("Heartbeat from connection [{}] ", connection)
val timestamp = clock()
val oldState = state.get
val latestTimestamp = oldState.timestamps.get(connection)
if (latestTimestamp.isEmpty) {
val newHistory = oldState.timestamps.get(connection) match {
case None
// this is heartbeat from a new connection
// add starter records for this new connection
val newState = oldState copy (
version = oldState.version + 1,
failureStats = oldState.failureStats + (connection -> FailureStats()),
intervalHistory = oldState.intervalHistory + (connection -> IndexedSeq.empty[Long]),
timestamps = oldState.timestamps + (connection -> timeMachine()),
explicitRemovals = oldState.explicitRemovals - connection)
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) heartbeat(connection) // recur
} else {
firstHeartbeat
case Some(latestTimestamp)
// this is a known connection
val timestamp = timeMachine()
val interval = timestamp - latestTimestamp.get
val newIntervalsForConnection = (oldState.intervalHistory.get(connection) match {
case Some(history) if history.size >= maxSampleSize
// reached max history, drop first interval
history drop 1
case Some(history) history
case _ IndexedSeq.empty[Long]
}) :+ interval
val newFailureStats =
if (newIntervalsForConnection.size > 1) {
val newMean: Double = newIntervalsForConnection.sum / newIntervalsForConnection.size.toDouble
val oldConnectionFailureStats = oldState.failureStats.get(connection).getOrElse {
throw new IllegalStateException("Can't calculate new failure statistics due to missing heartbeat history")
}
val deviationSum =
newIntervalsForConnection
.map(_.toDouble)
.foldLeft(0.0D)((x, y) x + (y - newMean))
val newVariance: Double = deviationSum / newIntervalsForConnection.size.toDouble
val newDeviation: Double = math.sqrt(newVariance)
val newFailureStats = oldConnectionFailureStats copy (mean = newMean, deviation = newDeviation, variance = newVariance)
oldState.failureStats + (connection -> newFailureStats)
} else {
oldState.failureStats
val interval = timestamp - latestTimestamp
oldState.history(connection) :+ interval
}
val newState = oldState copy (version = oldState.version + 1,
failureStats = newFailureStats,
intervalHistory = oldState.intervalHistory + (connection -> newIntervalsForConnection),
history = oldState.history + (connection -> newHistory),
timestamps = oldState.timestamps + (connection -> timestamp), // record new timestamp,
explicitRemovals = oldState.explicitRemovals - connection)
// 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/>
* The suspicion level of the accrual failure detector.
*
* 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.
* <p/>
* Implementations of 'Cumulative Distribution Function' for Exponential Distribution.
* For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597].
* considered healthy.
*/
def phi(connection: Address): Double = {
val oldState = state.get
val oldTimestamp = oldState.timestamps.get(connection)
val phi =
// if connection has been removed explicitly
if (oldState.explicitRemovals.contains(connection)) Double.MaxValue
else if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
else if (oldTimestamp.isEmpty) 0.0 // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
else {
val timestampDiff = timeMachine() - oldTimestamp.get
val timeDiff = clock() - oldTimestamp.get
val mean = oldState.failureStats.get(connection) match {
case Some(FailureStats(mean, _, _)) mean
case _ throw new IllegalStateException("Can't calculate Failure Detector Phi value for a node that have no heartbeat history")
val history = oldState.history(connection)
val mean = history.mean
val stdDeviation = ensureValidStdDeviation(history.stdDeviation)
val φ = phi(timeDiff, mean + acceptableHeartbeatPauseMillis, stdDeviation)
// FIXME change to debug log level, when failure detector is stable
if (φ > 1.0) log.info("Phi value [{}] for connection [{}], after [{} ms], based on [{}]",
φ, connection, timeDiff, "N(" + mean + ", " + stdDeviation + ")")
φ
}
}
if (mean == 0.0D) 0.0D
else PhiFactor * timestampDiff / mean
private[cluster] def phi(timeDiff: Long, mean: Double, stdDeviation: Double): Double = {
val cdf = cumulativeDistributionFunction(timeDiff, mean, stdDeviation)
-math.log10(1.0 - cdf)
}
// only log if PHI value is starting to get interesting
if (phi > 0.0D) log.debug("Node [{}] - Phi value [{}] and threshold [{}] for connection [{}] ", address, phi, threshold, connection)
phi
private val minStdDeviationMillis = minStdDeviation.toMillis
private def ensureValidStdDeviation(stdDeviation: Double): Double = math.max(stdDeviation, minStdDeviationMillis)
/**
* Cumulative distribution function for N(mean, stdDeviation) normal distribution.
* This is an approximation defined in β Mathematics Handbook.
*/
private[cluster] def cumulativeDistributionFunction(x: Double, mean: Double, stdDeviation: Double): Double = {
val y = (x - mean) / stdDeviation
// Cumulative distribution function for N(0, 1)
1.0 / (1.0 + math.exp(-y * (1.5976 + 0.070566 * y * y)))
}
/**
* Removes the heartbeat management for a connection.
*/
@tailrec
final def remove(connection: Address) {
final def remove(connection: Address): Unit = {
log.debug("Remove connection [{}] ", connection)
val oldState = state.get
if (oldState.failureStats.contains(connection)) {
if (oldState.history.contains(connection)) {
val newState = oldState copy (version = oldState.version + 1,
failureStats = oldState.failureStats - connection,
intervalHistory = oldState.intervalHistory - connection,
history = oldState.history - connection,
timestamps = oldState.timestamps - connection,
explicitRemovals = oldState.explicitRemovals + connection)
@ -182,3 +217,66 @@ class AccrualFailureDetector(
}
}
}
private[cluster] object HeartbeatHistory {
/**
* Create an empty HeartbeatHistory, without any history.
* Can only be used as starting point for appending intervals.
* The stats (mean, variance, stdDeviation) are not defined for
* for empty HeartbeatHistory, i.e. throws AritmeticException.
*/
def apply(maxSampleSize: Int): HeartbeatHistory = HeartbeatHistory(
maxSampleSize = maxSampleSize,
intervals = IndexedSeq.empty,
intervalSum = 0L,
squaredIntervalSum = 0L)
}
/**
* Holds the heartbeat statistics for a specific node Address.
* It is capped by the number of samples specified in `maxSampleSize`.
*
* The stats (mean, variance, stdDeviation) are not defined for
* for empty HeartbeatHistory, i.e. throws AritmeticException.
*/
private[cluster] case class HeartbeatHistory private (
maxSampleSize: Int,
intervals: IndexedSeq[Long],
intervalSum: Long,
squaredIntervalSum: Long) {
if (maxSampleSize < 1)
throw new IllegalArgumentException("maxSampleSize must be >= 1, got [%s]" format maxSampleSize)
if (intervalSum < 0L)
throw new IllegalArgumentException("intervalSum must be >= 0, got [%s]" format intervalSum)
if (squaredIntervalSum < 0L)
throw new IllegalArgumentException("squaredIntervalSum must be >= 0, got [%s]" format squaredIntervalSum)
def mean: Double = intervalSum.toDouble / intervals.size
def variance: Double = (squaredIntervalSum.toDouble / intervals.size) - (mean * mean)
def stdDeviation: Double = math.sqrt(variance)
@tailrec
final def :+(interval: Long): HeartbeatHistory = {
if (intervals.size < maxSampleSize)
HeartbeatHistory(
maxSampleSize,
intervals = intervals :+ interval,
intervalSum = intervalSum + interval,
squaredIntervalSum = squaredIntervalSum + pow2(interval))
else
dropOldest :+ interval // recur
}
private def dropOldest: HeartbeatHistory = HeartbeatHistory(
maxSampleSize,
intervals = intervals drop 1,
intervalSum = intervalSum - intervals.head,
squaredIntervalSum = squaredIntervalSum - pow2(intervals.head))
private def pow2(x: Long) = x * x
}

File diff suppressed because it is too large Load diff

View file

@ -13,17 +13,28 @@ import akka.actor.AddressFromURIString
class ClusterSettings(val config: Config, val systemName: String) {
import config._
val FailureDetectorThreshold = getInt("akka.cluster.failure-detector.threshold")
val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size")
val NodeToJoin: Option[Address] = getString("akka.cluster.node-to-join") match {
final val FailureDetectorThreshold = getDouble("akka.cluster.failure-detector.threshold")
final val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size")
final val FailureDetectorImplementationClass = getString("akka.cluster.failure-detector.implementation-class")
final val FailureDetectorMinStdDeviation: Duration =
Duration(getMilliseconds("akka.cluster.failure-detector.min-std-deviation"), MILLISECONDS)
final val FailureDetectorAcceptableHeartbeatPause: Duration =
Duration(getMilliseconds("akka.cluster.failure-detector.acceptable-heartbeat-pause"), MILLISECONDS)
final val NodeToJoin: Option[Address] = getString("akka.cluster.node-to-join") match {
case "" None
case AddressFromURIString(addr) Some(addr)
}
val PeriodicTasksInitialDelay = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
val GossipFrequency = Duration(getMilliseconds("akka.cluster.gossip-frequency"), MILLISECONDS)
val LeaderActionsFrequency = Duration(getMilliseconds("akka.cluster.leader-actions-frequency"), MILLISECONDS)
val UnreachableNodesReaperFrequency = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-frequency"), MILLISECONDS)
val NrOfGossipDaemons = getInt("akka.cluster.nr-of-gossip-daemons")
val NrOfDeputyNodes = getInt("akka.cluster.nr-of-deputy-nodes")
val AutoDown = getBoolean("akka.cluster.auto-down")
final val PeriodicTasksInitialDelay: Duration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
final val GossipInterval: Duration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
final val HeartbeatInterval: Duration = Duration(getMilliseconds("akka.cluster.heartbeat-interval"), MILLISECONDS)
final val LeaderActionsInterval: Duration = Duration(getMilliseconds("akka.cluster.leader-actions-interval"), MILLISECONDS)
final val UnreachableNodesReaperInterval: Duration = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS)
final val NrOfGossipDaemons: Int = getInt("akka.cluster.nr-of-gossip-daemons")
final val NrOfDeputyNodes: Int = getInt("akka.cluster.nr-of-deputy-nodes")
final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down")
final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS)
final val SchedulerTickDuration: Duration = Duration(getMilliseconds("akka.cluster.scheduler.tick-duration"), MILLISECONDS)
final val SchedulerTicksPerWheel: Int = getInt("akka.cluster.scheduler.ticks-per-wheel")
}

View file

@ -0,0 +1,28 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor.Address
/**
* Interface for Akka failure detectors.
*/
trait FailureDetector {
/**
* Returns true if the connection is considered to be up and healthy and returns false otherwise.
*/
def isAvailable(connection: Address): Boolean
/**
* Records a heartbeat for a connection.
*/
def heartbeat(connection: Address): Unit
/**
* Removes the heartbeat management for a connection.
*/
def remove(connection: Address): Unit
}

View file

@ -0,0 +1,51 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicLong
import akka.actor.Scheduler
import akka.util.Duration
/**
* INTERNAL API
*/
private[akka] object FixedRateTask {
def apply(scheduler: Scheduler, initalDelay: Duration, delay: Duration)(f: Unit): FixedRateTask = {
new FixedRateTask(scheduler, initalDelay, delay, new Runnable { def run(): Unit = f })
}
}
/**
* INTERNAL API
*
* Task to be scheduled periodically at a fixed rate, compensating, on average,
* for inaccuracy in scheduler. It will start when constructed, using the
* initialDelay.
*/
private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, delay: Duration, task: Runnable) extends Runnable {
private val delayNanos = delay.toNanos
private val cancelled = new AtomicBoolean(false)
private val counter = new AtomicLong(0L)
private val startTime = System.nanoTime + initalDelay.toNanos
scheduler.scheduleOnce(initalDelay, this)
def cancel(): Unit = cancelled.set(true)
override final def run(): Unit = if (!cancelled.get) try {
task.run()
} finally if (!cancelled.get) {
val nextTime = startTime + delayNanos * counter.incrementAndGet
// it's ok to schedule with negative duration, will run asap
val nextDelay = Duration(nextTime - System.nanoTime, TimeUnit.NANOSECONDS)
try {
scheduler.scheduleOnce(nextDelay, this)
} catch { case e: IllegalStateException /* will happen when scheduler is closed, nothing wrong */ }
}
}

View file

@ -19,7 +19,7 @@ class VectorClockException(message: String) extends AkkaException(message)
*/
trait Versioned[T] {
def version: VectorClock
def +(node: VectorClock.Node): T
def :+(node: VectorClock.Node): T
}
/**
@ -142,7 +142,7 @@ case class VectorClock(
/**
* Increment the version for the node passed as argument. Returns a new VectorClock.
*/
def +(node: Node): VectorClock = copy(versions = versions + (node -> Timestamp()))
def :+(node: Node): VectorClock = copy(versions = versions + (node -> Timestamp()))
/**
* Returns true if <code>this</code> and <code>that</code> are concurrent else false.

View file

@ -4,7 +4,6 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
@ -19,61 +18,52 @@ object ClientDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class ClientDowningNodeThatIsUnreachableMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec
class ClientDowningNodeThatIsUnreachableMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec
class ClientDowningNodeThatIsUnreachableMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec
class ClientDowningNodeThatIsUnreachableMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUnreachableSpec
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
abstract class ClientDowningNodeThatIsUnreachableSpec
extends MultiNodeSpec(ClientDowningNodeThatIsUnreachableMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender with BeforeAndAfter {
import ClientDowningNodeThatIsUnreachableMultiJvmSpec._
with MultiNodeClusterSpec {
override def initialParticipants = 4
import ClientDowningNodeThatIsUnreachableMultiJvmSpec._
"Client of a 4 node cluster" must {
"be able to DOWN a node that is UNREACHABLE (killed)" taggedAs LongRunningTest in {
val thirdAddress = address(third)
awaitClusterUp(first, second, third, fourth)
runOn(first) {
cluster.self
awaitUpConvergence(numberOfMembers = 4)
val thirdAddress = node(third).address
testConductor.enter("all-up")
// kill 'third' node
testConductor.shutdown(third, 0)
testConductor.removeNode(third)
markNodeAsUnavailable(thirdAddress)
// mark 'third' node as DOWN
cluster.down(thirdAddress)
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false)
testConductor.enter("await-completion")
}
runOn(third) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 4)
testConductor.enter("all-up")
enterBarrier("down-third-node")
}
runOn(second, fourth) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 4)
val thirdAddress = node(third).address
testConductor.enter("all-up")
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
testConductor.enter("await-completion")
}
enterBarrier("await-completion")
}
}
}

View file

@ -4,7 +4,6 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
@ -19,58 +18,50 @@ object ClientDowningNodeThatIsUpMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class ClientDowningNodeThatIsUpMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
abstract class ClientDowningNodeThatIsUpSpec
extends MultiNodeSpec(ClientDowningNodeThatIsUpMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender with BeforeAndAfter {
import ClientDowningNodeThatIsUpMultiJvmSpec._
with MultiNodeClusterSpec {
override def initialParticipants = 4
import ClientDowningNodeThatIsUpMultiJvmSpec._
"Client of a 4 node cluster" must {
"be able to DOWN a node that is UP (healthy and available)" taggedAs LongRunningTest in {
val thirdAddress = address(third)
awaitClusterUp(first, second, third, fourth)
runOn(first) {
cluster.self
awaitUpConvergence(numberOfMembers = 4)
val thirdAddress = node(third).address
testConductor.enter("all-up")
// mark 'third' node as DOWN
cluster.down(thirdAddress)
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
markNodeAsUnavailable(thirdAddress)
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false)
testConductor.enter("await-completion")
}
runOn(third) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 4)
testConductor.enter("all-up")
testConductor.enter("down-third-node")
testConductor.enter("await-completion")
enterBarrier("down-third-node")
}
runOn(second, fourth) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 4)
val thirdAddress = node(third).address
testConductor.enter("all-up")
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
testConductor.enter("await-completion")
}
enterBarrier("await-completion")
}
}
}

View file

@ -0,0 +1,63 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.util.duration._
import akka.testkit._
object ClusterAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold = 4")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class ClusterAccrualFailureDetectorMultiJvmNode1 extends ClusterAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
class ClusterAccrualFailureDetectorMultiJvmNode2 extends ClusterAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
class ClusterAccrualFailureDetectorMultiJvmNode3 extends ClusterAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
abstract class ClusterAccrualFailureDetectorSpec
extends MultiNodeSpec(ClusterAccrualFailureDetectorMultiJvmSpec)
with MultiNodeClusterSpec {
import ClusterAccrualFailureDetectorMultiJvmSpec._
"A heartbeat driven Failure Detector" must {
"receive heartbeats so that all member nodes in the cluster are marked 'available'" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
5.seconds.dilated.sleep // let them heartbeat
cluster.failureDetector.isAvailable(first) must be(true)
cluster.failureDetector.isAvailable(second) must be(true)
cluster.failureDetector.isAvailable(third) must be(true)
enterBarrier("after-1")
}
"mark node as 'unavailable' if a node in the cluster is shut down (and its heartbeats stops)" taggedAs LongRunningTest in {
runOn(first) {
testConductor.shutdown(third, 0)
}
enterBarrier("third-shutdown")
runOn(first, second) {
// remaning nodes should detect failure...
awaitCond(!cluster.failureDetector.isAvailable(third), 15.seconds)
// other connections still ok
cluster.failureDetector.isAvailable(first) must be(true)
cluster.failureDetector.isAvailable(second) must be(true)
}
enterBarrier("after-2")
}
}
}

View file

@ -0,0 +1,116 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
import akka.actor.Address
object ConvergenceMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold = 4")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class ConvergenceWithFailureDetectorPuppetMultiJvmNode1 extends ConvergenceSpec with FailureDetectorPuppetStrategy
class ConvergenceWithFailureDetectorPuppetMultiJvmNode2 extends ConvergenceSpec with FailureDetectorPuppetStrategy
class ConvergenceWithFailureDetectorPuppetMultiJvmNode3 extends ConvergenceSpec with FailureDetectorPuppetStrategy
class ConvergenceWithFailureDetectorPuppetMultiJvmNode4 extends ConvergenceSpec with FailureDetectorPuppetStrategy
class ConvergenceWithAccrualFailureDetectorMultiJvmNode1 extends ConvergenceSpec with AccrualFailureDetectorStrategy
class ConvergenceWithAccrualFailureDetectorMultiJvmNode2 extends ConvergenceSpec with AccrualFailureDetectorStrategy
class ConvergenceWithAccrualFailureDetectorMultiJvmNode3 extends ConvergenceSpec with AccrualFailureDetectorStrategy
class ConvergenceWithAccrualFailureDetectorMultiJvmNode4 extends ConvergenceSpec with AccrualFailureDetectorStrategy
abstract class ConvergenceSpec
extends MultiNodeSpec(ConvergenceMultiJvmSpec)
with MultiNodeClusterSpec {
import ConvergenceMultiJvmSpec._
"A cluster of 3 members" must {
"reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(fourth) {
// doesn't join immediately
}
enterBarrier("after-1")
}
"not reach convergence while any nodes are unreachable" taggedAs LongRunningTest in {
val thirdAddress = address(third)
enterBarrier("before-shutdown")
runOn(first) {
// kill 'third' node
testConductor.shutdown(third, 0)
markNodeAsUnavailable(thirdAddress)
}
runOn(first, second) {
within(28 seconds) {
// third becomes unreachable
awaitCond(cluster.latestGossip.overview.unreachable.size == 1)
awaitCond(cluster.latestGossip.members.size == 2)
awaitCond(cluster.latestGossip.members.forall(_.status == MemberStatus.Up))
awaitSeenSameState(first, second)
// still one unreachable
cluster.latestGossip.overview.unreachable.size must be(1)
cluster.latestGossip.overview.unreachable.head.address must be(thirdAddress)
// and therefore no convergence
cluster.convergence.isDefined must be(false)
}
}
enterBarrier("after-2")
}
"not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest in {
runOn(fourth) {
// try to join
cluster.join(first)
}
def memberStatus(address: Address): Option[MemberStatus] =
cluster.latestGossip.members.collectFirst { case m if m.address == address m.status }
def assertNotMovedUp: Unit = {
within(20 seconds) {
awaitCond(cluster.latestGossip.members.size == 3)
awaitSeenSameState(first, second, fourth)
memberStatus(first) must be(Some(MemberStatus.Up))
memberStatus(second) must be(Some(MemberStatus.Up))
// leader is not allowed to move the new node to Up
memberStatus(fourth) must be(Some(MemberStatus.Joining))
// still no convergence
cluster.convergence.isDefined must be(false)
}
}
runOn(first, second, fourth) {
for (n 1 to 5) {
log.debug("assertNotMovedUp#" + n)
assertNotMovedUp
// wait and then check again
1.second.dilated.sleep
}
}
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,61 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor.Address
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
/**
* Base trait for all failure detector strategies.
*/
trait FailureDetectorStrategy {
/**
* Get or create the FailureDetector to be used in the cluster node.
* To be defined by subclass.
*/
def failureDetector: FailureDetector
/**
* Marks a node as available in the failure detector.
* To be defined by subclass.
*/
def markNodeAsAvailable(address: Address): Unit
/**
* Marks a node as unavailable in the failure detector.
* To be defined by subclass.
*/
def markNodeAsUnavailable(address: Address): Unit
}
/**
* Defines a FailureDetectorPuppet-based FailureDetectorStrategy.
*/
trait FailureDetectorPuppetStrategy extends FailureDetectorStrategy { self: MultiNodeSpec
/**
* The puppet instance. Separated from 'failureDetector' field so we don't have to cast when using the puppet specific methods.
*/
private val puppet = new FailureDetectorPuppet(system)
override def failureDetector: FailureDetector = puppet
override def markNodeAsAvailable(address: Address): Unit = puppet markNodeAsAvailable address
override def markNodeAsUnavailable(address: Address): Unit = puppet markNodeAsUnavailable address
}
/**
* Defines a AccrualFailureDetector-based FailureDetectorStrategy.
*/
trait AccrualFailureDetectorStrategy extends FailureDetectorStrategy { self: MultiNodeSpec
override val failureDetector: FailureDetector = new AccrualFailureDetector(system, new ClusterSettings(system.settings.config, system.name))
override def markNodeAsAvailable(address: Address): Unit = ()
override def markNodeAsUnavailable(address: Address): Unit = ()
}

View file

@ -1,74 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import org.scalatest.BeforeAndAfter
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.util.duration._
import akka.testkit._
object GossipingAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold=4")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class GossipingAccrualFailureDetectorMultiJvmNode1 extends GossipingAccrualFailureDetectorSpec
class GossipingAccrualFailureDetectorMultiJvmNode2 extends GossipingAccrualFailureDetectorSpec
class GossipingAccrualFailureDetectorMultiJvmNode3 extends GossipingAccrualFailureDetectorSpec
abstract class GossipingAccrualFailureDetectorSpec extends MultiNodeSpec(GossipingAccrualFailureDetectorMultiJvmSpec)
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
import GossipingAccrualFailureDetectorMultiJvmSpec._
override def initialParticipants = 3
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
after {
testConductor.enter("after")
}
"A Gossip-driven Failure Detector" must {
"receive gossip heartbeats so that all member nodes in the cluster are marked 'available'" taggedAs LongRunningTest in {
// make sure that the node-to-join is started before other join
runOn(first) {
cluster.self
}
testConductor.enter("first-started")
cluster.join(firstAddress)
5.seconds.dilated.sleep // let them gossip
cluster.failureDetector.isAvailable(firstAddress) must be(true)
cluster.failureDetector.isAvailable(secondAddress) must be(true)
cluster.failureDetector.isAvailable(thirdAddress) must be(true)
}
"mark node as 'unavailable' if a node in the cluster is shut down (and its heartbeats stops)" taggedAs LongRunningTest in {
runOn(first) {
testConductor.shutdown(third, 0)
testConductor.removeNode(third)
}
runOn(first, second) {
// remaning nodes should detect failure...
awaitCond(!cluster.failureDetector.isAvailable(thirdAddress), 10.seconds)
// other connections still ok
cluster.failureDetector.isAvailable(firstAddress) must be(true)
cluster.failureDetector.isAvailable(secondAddress) must be(true)
}
}
}
}

View file

@ -0,0 +1,65 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
import akka.util.Deadline
object JoinInProgressMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster {
# simulate delay in gossip by turning it off
gossip-interval = 300 s
failure-detector {
threshold = 4
acceptable-heartbeat-pause = 1 second
}
}""") // increase the leader action task interval
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class JoinInProgressMultiJvmNode1 extends JoinInProgressSpec with AccrualFailureDetectorStrategy
class JoinInProgressMultiJvmNode2 extends JoinInProgressSpec with AccrualFailureDetectorStrategy
abstract class JoinInProgressSpec
extends MultiNodeSpec(JoinInProgressMultiJvmSpec)
with MultiNodeClusterSpec {
import JoinInProgressMultiJvmSpec._
"A cluster node" must {
"send heartbeats immediately when joining to avoid false failure detection due to delayed gossip" taggedAs LongRunningTest in {
runOn(first) {
startClusterNode()
}
enterBarrier("first-started")
runOn(second) {
cluster.join(first)
}
runOn(first) {
val until = Deadline.now + 5.seconds
while (!until.isOverdue) {
200.millis.sleep
cluster.failureDetector.isAvailable(second) must be(true)
}
}
enterBarrier("after")
}
}
}

View file

@ -18,42 +18,38 @@ object JoinTwoClustersMultiJvmSpec extends MultiNodeConfig {
val c2 = role("c2")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec
class JoinTwoClustersMultiJvmNode2 extends JoinTwoClustersSpec
class JoinTwoClustersMultiJvmNode3 extends JoinTwoClustersSpec
class JoinTwoClustersMultiJvmNode4 extends JoinTwoClustersSpec
class JoinTwoClustersMultiJvmNode5 extends JoinTwoClustersSpec
class JoinTwoClustersMultiJvmNode6 extends JoinTwoClustersSpec
class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
class JoinTwoClustersMultiJvmNode2 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
class JoinTwoClustersMultiJvmNode3 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
class JoinTwoClustersMultiJvmNode4 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
class JoinTwoClustersMultiJvmNode5 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
class JoinTwoClustersMultiJvmNode6 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
abstract class JoinTwoClustersSpec
extends MultiNodeSpec(JoinTwoClustersMultiJvmSpec)
with MultiNodeClusterSpec {
abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender {
import JoinTwoClustersMultiJvmSpec._
override def initialParticipants = 6
lazy val a1Address = node(a1).address
lazy val b1Address = node(b1).address
lazy val c1Address = node(c1).address
"Three different clusters (A, B and C)" must {
"be able to 'elect' a single leader after joining (A -> B)" taggedAs LongRunningTest in {
// make sure that the node-to-join is started before other join
runOn(a1, b1, c1) {
cluster.self
startClusterNode()
}
testConductor.enter("first-started")
enterBarrier("first-started")
runOn(a1, a2) {
cluster.join(a1Address)
cluster.join(a1)
}
runOn(b1, b2) {
cluster.join(b1Address)
cluster.join(b1)
}
runOn(c1, c2) {
cluster.join(c1Address)
cluster.join(c1)
}
awaitUpConvergence(numberOfMembers = 2)
@ -62,10 +58,10 @@ abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvm
assertLeader(b1, b2)
assertLeader(c1, c2)
testConductor.enter("two-members")
enterBarrier("two-members")
runOn(b2) {
cluster.join(a1Address)
cluster.join(a1)
}
runOn(a1, a2, b1, b2) {
@ -75,22 +71,20 @@ abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvm
assertLeader(a1, a2, b1, b2)
assertLeader(c1, c2)
testConductor.enter("four-members")
enterBarrier("four-members")
}
"be able to 'elect' a single leader after joining (C -> A + B)" taggedAs LongRunningTest in {
runOn(b2) {
cluster.join(c1Address)
cluster.join(c1)
}
awaitUpConvergence(numberOfMembers = 6)
assertLeader(a1, a2, b1, b2, c1, c2)
testConductor.enter("six-members")
enterBarrier("six-members")
}
}
}

View file

@ -4,11 +4,10 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.actor.Address
import akka.actor._
import akka.util.duration._
object LeaderDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
@ -18,108 +17,86 @@ object LeaderDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
val fourth = role("fourth")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("""
akka.cluster {
auto-down = on
failure-detector.threshold = 4
}
""")).
withFallback(ConfigFactory.parseString("akka.cluster.auto-down = on")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class LeaderDowningNodeThatIsUnreachableMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec
class LeaderDowningNodeThatIsUnreachableMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec
class LeaderDowningNodeThatIsUnreachableMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec
class LeaderDowningNodeThatIsUnreachableMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
class LeaderDowningNodeThatIsUnreachableSpec
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
abstract class LeaderDowningNodeThatIsUnreachableSpec
extends MultiNodeSpec(LeaderDowningNodeThatIsUnreachableMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender with BeforeAndAfter {
import LeaderDowningNodeThatIsUnreachableMultiJvmSpec._
with MultiNodeClusterSpec {
override def initialParticipants = 4
import LeaderDowningNodeThatIsUnreachableMultiJvmSpec._
"The Leader in a 4 node cluster" must {
"be able to DOWN a 'last' node that is UNREACHABLE" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third, fourth)
val fourthAddress = address(fourth)
runOn(first) {
cluster.self
awaitUpConvergence(numberOfMembers = 4)
val fourthAddress = node(fourth).address
testConductor.enter("all-up")
// kill 'fourth' node
testConductor.shutdown(fourth, 0)
testConductor.removeNode(fourth)
testConductor.enter("down-fourth-node")
enterBarrier("down-fourth-node")
// mark the node as unreachable in the failure detector
markNodeAsUnavailable(fourthAddress)
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
testConductor.enter("await-completion")
}
runOn(fourth) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 4)
testConductor.enter("all-up")
enterBarrier("down-fourth-node")
}
runOn(second, third) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 4)
val fourthAddress = node(fourth).address
testConductor.enter("all-up")
testConductor.enter("down-fourth-node")
enterBarrier("down-fourth-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
testConductor.enter("await-completion")
}
enterBarrier("await-completion-1")
}
"be able to DOWN a 'middle' node that is UNREACHABLE" taggedAs LongRunningTest in {
val secondAddress = address(second)
enterBarrier("before-down-second-node")
runOn(first) {
cluster.self
awaitUpConvergence(numberOfMembers = 3)
val secondAddress = node(second).address
testConductor.enter("all-up")
// kill 'second' node
testConductor.shutdown(second, 0)
testConductor.removeNode(second)
testConductor.enter("down-second-node")
enterBarrier("down-second-node")
// mark the node as unreachable in the failure detector
markNodeAsUnavailable(secondAddress)
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
testConductor.enter("await-completion")
}
runOn(second) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 3)
testConductor.enter("all-up")
enterBarrier("down-second-node")
}
runOn(third) {
cluster.join(node(first).address)
awaitUpConvergence(numberOfMembers = 3)
val secondAddress = node(second).address
testConductor.enter("all-up")
testConductor.enter("down-second-node")
enterBarrier("down-second-node")
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds)
testConductor.enter("await-completion")
}
enterBarrier("await-completion-2")
}
}
}

View file

@ -17,88 +17,92 @@ object LeaderElectionMultiJvmSpec extends MultiNodeConfig {
val fourth = role("fourth")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class LeaderElectionMultiJvmNode1 extends LeaderElectionSpec
class LeaderElectionMultiJvmNode2 extends LeaderElectionSpec
class LeaderElectionMultiJvmNode3 extends LeaderElectionSpec
class LeaderElectionMultiJvmNode4 extends LeaderElectionSpec
class LeaderElectionMultiJvmNode5 extends LeaderElectionSpec
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode1 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode2 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode3 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode4 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode5 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode1 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode2 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode3 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode4 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode5 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
abstract class LeaderElectionSpec
extends MultiNodeSpec(LeaderElectionMultiJvmSpec)
with MultiNodeClusterSpec {
abstract class LeaderElectionSpec extends MultiNodeSpec(LeaderElectionMultiJvmSpec) with MultiNodeClusterSpec {
import LeaderElectionMultiJvmSpec._
override def initialParticipants = 5
lazy val firstAddress = node(first).address
// sorted in the order used by the cluster
lazy val roles = Seq(first, second, third, fourth).sorted
lazy val sortedRoles = Seq(first, second, third, fourth).sorted
"A cluster of four nodes" must {
"be able to 'elect' a single leader" taggedAs LongRunningTest in {
// make sure that the node-to-join is started before other join
runOn(first) {
cluster.self
}
testConductor.enter("first-started")
awaitClusterUp(first, second, third, fourth)
if (myself != controller) {
cluster.join(firstAddress)
awaitUpConvergence(numberOfMembers = roles.size)
cluster.isLeader must be(myself == roles.head)
cluster.isLeader must be(myself == sortedRoles.head)
assertLeaderIn(sortedRoles)
}
testConductor.enter("after")
enterBarrier("after-1")
}
def shutdownLeaderAndVerifyNewLeader(alreadyShutdown: Int): Unit = {
val currentRoles = roles.drop(alreadyShutdown)
val currentRoles = sortedRoles.drop(alreadyShutdown)
currentRoles.size must be >= (2)
val leader = currentRoles.head
val aUser = currentRoles.last
val remainingRoles = currentRoles.tail
myself match {
case `controller`
testConductor.enter("before-shutdown")
val leaderAddress = address(leader)
enterBarrier("before-shutdown")
testConductor.shutdown(leader, 0)
testConductor.removeNode(leader)
testConductor.enter("after-shutdown", "after-down", "completed")
enterBarrier("after-shutdown", "after-down", "completed")
markNodeAsUnavailable(leaderAddress)
case `leader`
testConductor.enter("before-shutdown")
enterBarrier("before-shutdown", "after-shutdown")
// this node will be shutdown by the controller and doesn't participate in more barriers
case `aUser`
val leaderAddress = node(leader).address
testConductor.enter("before-shutdown", "after-shutdown")
val leaderAddress = address(leader)
enterBarrier("before-shutdown", "after-shutdown")
// user marks the shutdown leader as DOWN
cluster.down(leaderAddress)
testConductor.enter("after-down", "completed")
enterBarrier("after-down", "completed")
markNodeAsUnavailable(leaderAddress)
case _ if currentRoles.tail.contains(myself)
case _ if remainingRoles.contains(myself)
// remaining cluster nodes, not shutdown
testConductor.enter("before-shutdown", "after-shutdown", "after-down")
enterBarrier("before-shutdown", "after-shutdown", "after-down")
awaitUpConvergence(currentRoles.size - 1)
val nextExpectedLeader = currentRoles.tail.head
val nextExpectedLeader = remainingRoles.head
cluster.isLeader must be(myself == nextExpectedLeader)
assertLeaderIn(remainingRoles)
testConductor.enter("completed")
enterBarrier("completed")
}
}
"be able to 're-elect' a single leader after leader has left" taggedAs LongRunningTest in {
shutdownLeaderAndVerifyNewLeader(alreadyShutdown = 0)
enterBarrier("after-2")
}
"be able to 're-elect' a single leader after leader has left (again)" taggedAs LongRunningTest in {
shutdownLeaderAndVerifyNewLeader(alreadyShutdown = 1)
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,85 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object LeaderLeavingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster {
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state
unreachable-nodes-reaper-interval = 30 s
}""")
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class LeaderLeavingMultiJvmNode1 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy
class LeaderLeavingMultiJvmNode2 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy
class LeaderLeavingMultiJvmNode3 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy
abstract class LeaderLeavingSpec
extends MultiNodeSpec(LeaderLeavingMultiJvmSpec)
with MultiNodeClusterSpec {
import LeaderLeavingMultiJvmSpec._
val leaderHandoffWaitingTime = 30.seconds.dilated
"A LEADER that is LEAVING" must {
"be moved to LEAVING, then to EXITING, then to REMOVED, then be shut down and then a new LEADER should be elected" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
val oldLeaderAddress = cluster.leader
if (cluster.isLeader) {
cluster.leave(oldLeaderAddress)
enterBarrier("leader-left")
// verify that a NEW LEADER have taken over
awaitCond(!cluster.isLeader)
// verify that the LEADER is shut down
awaitCond(!cluster.isRunning, 30.seconds.dilated)
// verify that the LEADER is REMOVED
awaitCond(cluster.status == MemberStatus.Removed)
} else {
enterBarrier("leader-left")
// verify that the LEADER is LEAVING
awaitCond(cluster.latestGossip.members.exists(m m.status == MemberStatus.Leaving && m.address == oldLeaderAddress), leaderHandoffWaitingTime) // wait on LEAVING
// verify that the LEADER is EXITING
awaitCond(cluster.latestGossip.members.exists(m m.status == MemberStatus.Exiting && m.address == oldLeaderAddress), leaderHandoffWaitingTime) // wait on EXITING
// verify that the LEADER is no longer part of the 'members' set
awaitCond(cluster.latestGossip.members.forall(_.address != oldLeaderAddress), leaderHandoffWaitingTime)
// verify that the LEADER is not part of the 'unreachable' set
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != oldLeaderAddress), leaderHandoffWaitingTime)
// verify that we have a new LEADER
awaitCond(cluster.leader != oldLeaderAddress, leaderHandoffWaitingTime)
}
enterBarrier("finished")
}
}
}

View file

@ -0,0 +1,69 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import org.scalatest.BeforeAndAfter
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object MembershipChangeListenerExitingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster {
leader-actions-interval = 5 s # increase the leader action task interval
unreachable-nodes-reaper-interval = 300 s # turn "off" reaping to unreachable node set
}
""")
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class MembershipChangeListenerExitingMultiJvmNode1 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerExitingMultiJvmNode2 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerExitingMultiJvmNode3 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy
abstract class MembershipChangeListenerExitingSpec
extends MultiNodeSpec(MembershipChangeListenerExitingMultiJvmSpec)
with MultiNodeClusterSpec {
import MembershipChangeListenerExitingMultiJvmSpec._
"A registered MembershipChangeListener" must {
"be notified when new node is EXITING" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
enterBarrier("registered-listener")
cluster.leave(second)
}
runOn(second) {
enterBarrier("registered-listener")
}
runOn(third) {
val exitingLatch = TestLatch()
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.size == 3 && members.exists(m m.address == address(second) && m.status == MemberStatus.Exiting))
exitingLatch.countDown()
}
})
enterBarrier("registered-listener")
exitingLatch.await
}
enterBarrier("finished")
}
}
}

View file

@ -0,0 +1,60 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object MembershipChangeListenerJoinMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval to allow time checking for JOIN before leader moves it to UP
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class MembershipChangeListenerJoinMultiJvmNode1 extends MembershipChangeListenerJoinSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerJoinMultiJvmNode2 extends MembershipChangeListenerJoinSpec with FailureDetectorPuppetStrategy
abstract class MembershipChangeListenerJoinSpec
extends MultiNodeSpec(MembershipChangeListenerJoinMultiJvmSpec)
with MultiNodeClusterSpec {
import MembershipChangeListenerJoinMultiJvmSpec._
"A registered MembershipChangeListener" must {
"be notified when new node is JOINING" taggedAs LongRunningTest in {
runOn(first) {
val joinLatch = TestLatch()
val expectedAddresses = Set(first, second) map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.map(_.address) == expectedAddresses && members.exists(_.status == MemberStatus.Joining))
joinLatch.countDown()
}
})
enterBarrier("registered-listener")
joinLatch.await
}
runOn(second) {
enterBarrier("registered-listener")
cluster.join(first)
}
awaitUpConvergence(2)
enterBarrier("after")
}
}
}

View file

@ -0,0 +1,69 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import org.scalatest.BeforeAndAfter
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.actor.Address
object MembershipChangeListenerLeavingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster.leader-actions-interval = 5 s
akka.cluster.unreachable-nodes-reaper-interval = 300 s # turn "off"
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
class MembershipChangeListenerLeavingMultiJvmNode1 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerLeavingMultiJvmNode2 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerLeavingMultiJvmNode3 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy
abstract class MembershipChangeListenerLeavingSpec
extends MultiNodeSpec(MembershipChangeListenerLeavingMultiJvmSpec)
with MultiNodeClusterSpec {
import MembershipChangeListenerLeavingMultiJvmSpec._
"A registered MembershipChangeListener" must {
"be notified when new node is LEAVING" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
enterBarrier("registered-listener")
cluster.leave(second)
}
runOn(second) {
enterBarrier("registered-listener")
}
runOn(third) {
val latch = TestLatch()
val expectedAddresses = Set(first, second, third) map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.map(_.address) == expectedAddresses &&
members.exists(m m.address == address(second) && m.status == MemberStatus.Leaving))
latch.countDown()
}
})
enterBarrier("registered-listener")
latch.await
}
enterBarrier("finished")
}
}
}

View file

@ -1,83 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import org.scalatest.BeforeAndAfter
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
object MembershipChangeListenerMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class MembershipChangeListenerMultiJvmNode1 extends MembershipChangeListenerSpec
class MembershipChangeListenerMultiJvmNode2 extends MembershipChangeListenerSpec
class MembershipChangeListenerMultiJvmNode3 extends MembershipChangeListenerSpec
abstract class MembershipChangeListenerSpec extends MultiNodeSpec(MembershipChangeListenerMultiJvmSpec)
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
import MembershipChangeListenerMultiJvmSpec._
override def initialParticipants = 3
after {
testConductor.enter("after")
}
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
"A set of connected cluster systems" must {
"(when two systems) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
// make sure that the node-to-join is started before other join
runOn(first) {
cluster.self
}
testConductor.enter("first-started")
runOn(first, second) {
cluster.join(firstAddress)
val latch = TestLatch()
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.size == 2 && members.forall(_.status == MemberStatus.Up))
latch.countDown()
}
})
latch.await
cluster.convergence.isDefined must be(true)
}
}
"(when three systems) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
runOn(third) {
cluster.join(firstAddress)
}
val latch = TestLatch()
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.size == 3 && members.forall(_.status == MemberStatus.Up))
latch.countDown()
}
})
latch.await
cluster.convergence.isDefined must be(true)
}
}
}

View file

@ -0,0 +1,78 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
object MembershipChangeListenerUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class MembershipChangeListenerUpMultiJvmNode1 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerUpMultiJvmNode2 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy
class MembershipChangeListenerUpMultiJvmNode3 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy
abstract class MembershipChangeListenerUpSpec
extends MultiNodeSpec(MembershipChangeListenerUpMultiJvmSpec)
with MultiNodeClusterSpec {
import MembershipChangeListenerUpMultiJvmSpec._
"A set of connected cluster systems" must {
"(when two nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
awaitClusterUp(first)
runOn(first, second) {
val latch = TestLatch()
val expectedAddresses = Set(first, second) map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.map(_.address) == expectedAddresses && members.forall(_.status == MemberStatus.Up))
latch.countDown()
}
})
enterBarrier("listener-1-registered")
cluster.join(first)
latch.await
}
runOn(third) {
enterBarrier("listener-1-registered")
}
enterBarrier("after-1")
}
"(when three nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
val latch = TestLatch()
val expectedAddresses = Set(first, second, third) map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.map(_.address) == expectedAddresses && members.forall(_.status == MemberStatus.Up))
latch.countDown()
}
})
enterBarrier("listener-2-registered")
runOn(third) {
cluster.join(first)
}
latch.await
enterBarrier("after-2")
}
}
}

View file

@ -5,21 +5,28 @@ package akka.cluster
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor.Address
import akka.actor.{ Address, ExtendedActorSystem }
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
import akka.util.Duration
import org.scalatest.Suite
import org.scalatest.TestFailedException
import java.util.concurrent.ConcurrentHashMap
import akka.actor.ActorPath
import akka.actor.RootActorPath
object MultiNodeClusterSpec {
def clusterConfig: Config = ConfigFactory.parseString("""
akka.cluster {
auto-down = off
gossip-frequency = 200 ms
leader-actions-frequency = 200 ms
unreachable-nodes-reaper-frequency = 200 ms
gossip-interval = 200 ms
heartbeat-interval = 400 ms
leader-actions-interval = 200 ms
unreachable-nodes-reaper-interval = 200 ms
periodic-tasks-initial-delay = 300 ms
nr-of-deputy-nodes = 2
}
akka.test {
single-expect-default = 5 s
@ -27,9 +34,96 @@ object MultiNodeClusterSpec {
""")
}
trait MultiNodeClusterSpec { self: MultiNodeSpec
trait MultiNodeClusterSpec extends FailureDetectorStrategy with Suite { self: MultiNodeSpec
def cluster: Cluster = Cluster(system)
override def initialParticipants = roles.size
private val cachedAddresses = new ConcurrentHashMap[RoleName, Address]
/**
* Lookup the Address for the role.
*
* Implicit conversion from RoleName to Address.
*
* It is cached, which has the implication that stopping
* and then restarting a role (jvm) with another address is not
* supported.
*/
implicit def address(role: RoleName): Address = {
cachedAddresses.get(role) match {
case null
val address = node(role).address
cachedAddresses.put(role, address)
address
case address address
}
}
// Cluster tests are written so that if previous step (test method) failed
// it will most likely not be possible to run next step. This ensures
// fail fast of steps after the first failure.
private var failed = false
override protected def withFixture(test: NoArgTest): Unit = try {
if (failed) {
val e = new TestFailedException("Previous step failed", 0)
// short stack trace
e.setStackTrace(e.getStackTrace.take(1))
throw e
}
super.withFixture(test)
} catch {
case t
failed = true
throw t
}
/**
* The cluster node instance. Needs to be lazily created.
*/
private lazy val clusterNode = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector)
/**
* Get the cluster node to use.
*/
def cluster: Cluster = clusterNode
/**
* Use this method instead of 'cluster.self'
* for the initial startup of the cluster node.
*/
def startClusterNode(): Unit = cluster.self
/**
* Initialize the cluster with the specified member
* nodes (roles). First node will be started first
* and others will join the first.
*/
def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.toSeq)
/**
* Initialize the cluster of the specified member
* nodes (roles) and wait until all joined and `Up`.
* First node will be started first and others will join
* the first.
*/
def awaitClusterUp(roles: RoleName*): Unit = {
awaitStartCluster(true, roles.toSeq)
}
private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = {
runOn(roles.head) {
// make sure that the node-to-join is started before other join
startClusterNode()
}
enterBarrier(roles.head.name + "-started")
if (roles.tail.contains(myself)) {
cluster.join(roles.head)
}
if (upConvergence && roles.contains(myself)) {
awaitUpConvergence(numberOfMembers = roles.length)
}
enterBarrier(roles.map(_.name).mkString("-") + "-joined")
}
/**
* Assert that the member addresses match the expected addresses in the
@ -42,15 +136,20 @@ trait MultiNodeClusterSpec { self: MultiNodeSpec ⇒
expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) members(i).address must be(a) }
}
def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) {
assertLeaderIn(nodesInCluster)
}
/**
* Assert that the cluster has elected the correct leader
* out of all nodes in the cluster. First
* member in the cluster ring is expected leader.
*/
def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) {
def assertLeaderIn(nodesInCluster: Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) {
nodesInCluster.length must not be (0)
val expectedLeader = roleOfLeader(nodesInCluster)
cluster.isLeader must be(ifNode(expectedLeader)(true)(false))
cluster.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving))
}
/**
@ -71,6 +170,17 @@ trait MultiNodeClusterSpec { self: MultiNodeSpec ⇒
}
}
/**
* Wait until the specified nodes have seen the same gossip overview.
*/
def awaitSeenSameState(addresses: Address*): Unit = {
awaitCond {
val seen = cluster.latestGossip.overview.seen
val seenVectorClocks = addresses.flatMap(seen.get(_))
seenVectorClocks.size == addresses.size && seenVectorClocks.toSet.size == 1
}
}
def roleOfLeader(nodesInCluster: Seq[RoleName]): RoleName = {
nodesInCluster.length must not be (0)
nodesInCluster.sorted.head
@ -81,10 +191,9 @@ trait MultiNodeClusterSpec { self: MultiNodeSpec ⇒
*/
implicit val clusterOrdering: Ordering[RoleName] = new Ordering[RoleName] {
import Member.addressOrdering
def compare(x: RoleName, y: RoleName) = addressOrdering.compare(node(x).address, node(y).address)
def compare(x: RoleName, y: RoleName) = addressOrdering.compare(address(x), address(y))
}
def roleName(address: Address): Option[RoleName] = {
testConductor.getNodes.await.find(node(_).address == address)
}
def roleName(addr: Address): Option[RoleName] = roles.find(address(_) == addr)
}

View file

@ -0,0 +1,48 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object NodeJoinMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class NodeJoinMultiJvmNode1 extends NodeJoinSpec with FailureDetectorPuppetStrategy
class NodeJoinMultiJvmNode2 extends NodeJoinSpec with FailureDetectorPuppetStrategy
abstract class NodeJoinSpec
extends MultiNodeSpec(NodeJoinMultiJvmSpec)
with MultiNodeClusterSpec {
import NodeJoinMultiJvmSpec._
"A cluster node" must {
"join another cluster and get status JOINING - when sending a 'Join' command" taggedAs LongRunningTest in {
runOn(first) {
startClusterNode()
}
runOn(second) {
cluster.join(first)
}
awaitCond(cluster.latestGossip.members.exists { member member.address == address(second) && member.status == MemberStatus.Joining })
enterBarrier("after")
}
}
}

View file

@ -0,0 +1,61 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
abstract class NodeLeavingAndExitingAndBeingRemovedSpec
extends MultiNodeSpec(NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec)
with MultiNodeClusterSpec {
import NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec._
val reaperWaitingTime = 30.seconds.dilated
"A node that is LEAVING a non-singleton cluster" must {
"eventually set to REMOVED by the reaper, and removed from membership ring and seen table" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
cluster.leave(second)
}
enterBarrier("second-left")
runOn(first, third) {
// verify that the 'second' node is no longer part of the 'members' set
awaitCond(cluster.latestGossip.members.forall(_.address != address(second)), reaperWaitingTime)
// verify that the 'second' node is not part of the 'unreachable' set
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != address(second)), reaperWaitingTime)
}
runOn(second) {
// verify that the second node is shut down and has status REMOVED
awaitCond(!cluster.isRunning, reaperWaitingTime)
awaitCond(cluster.status == MemberStatus.Removed, reaperWaitingTime)
}
enterBarrier("finished")
}
}
}

View file

@ -0,0 +1,70 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster {
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state
unreachable-nodes-reaper-interval = 300 s # turn "off"
}
""")
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class NodeLeavingAndExitingMultiJvmNode1 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy
class NodeLeavingAndExitingMultiJvmNode2 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy
class NodeLeavingAndExitingMultiJvmNode3 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy
abstract class NodeLeavingAndExitingSpec
extends MultiNodeSpec(NodeLeavingAndExitingMultiJvmSpec)
with MultiNodeClusterSpec {
import NodeLeavingAndExitingMultiJvmSpec._
"A node that is LEAVING a non-singleton cluster" must {
"be moved to EXITING by the leader" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
cluster.leave(second)
}
enterBarrier("second-left")
runOn(first, third) {
// 1. Verify that 'second' node is set to LEAVING
// We have set the 'leader-actions-interval' to 5 seconds to make sure that we get a
// chance to test the LEAVING state before the leader moves the node to EXITING
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving)) // wait on LEAVING
val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving) // verify node that left
hasLeft must be('defined)
hasLeft.get.address must be(address(second))
// 2. Verify that 'second' node is set to EXITING
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Exiting)) // wait on EXITING
val hasExited = cluster.latestGossip.members.find(_.status == MemberStatus.Exiting) // verify node that exited
hasExited must be('defined)
hasExited.get.address must be(address(second))
}
enterBarrier("finished")
}
}
}

View file

@ -0,0 +1,55 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
object NodeLeavingMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("akka.cluster.unreachable-nodes-reaper-frequency = 30 s"))
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeLeavingMultiJvmNode1 extends NodeLeavingSpec with FailureDetectorPuppetStrategy
class NodeLeavingMultiJvmNode2 extends NodeLeavingSpec with FailureDetectorPuppetStrategy
class NodeLeavingMultiJvmNode3 extends NodeLeavingSpec with FailureDetectorPuppetStrategy
abstract class NodeLeavingSpec
extends MultiNodeSpec(NodeLeavingMultiJvmSpec)
with MultiNodeClusterSpec {
import NodeLeavingMultiJvmSpec._
"A node that is LEAVING a non-singleton cluster" must {
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
cluster.leave(second)
}
enterBarrier("second-left")
runOn(first, third) {
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving))
val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving)
hasLeft must be('defined)
hasLeft.get.address must be(address(second))
}
enterBarrier("finished")
}
}
}

View file

@ -4,7 +4,6 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
@ -15,62 +14,55 @@ object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
val third = role("third")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeMembershipMultiJvmNode1 extends NodeMembershipSpec
class NodeMembershipMultiJvmNode2 extends NodeMembershipSpec
class NodeMembershipMultiJvmNode3 extends NodeMembershipSpec
class NodeMembershipMultiJvmNode1 extends NodeMembershipSpec with FailureDetectorPuppetStrategy
class NodeMembershipMultiJvmNode2 extends NodeMembershipSpec with FailureDetectorPuppetStrategy
class NodeMembershipMultiJvmNode3 extends NodeMembershipSpec with FailureDetectorPuppetStrategy
abstract class NodeMembershipSpec
extends MultiNodeSpec(NodeMembershipMultiJvmSpec)
with MultiNodeClusterSpec {
abstract class NodeMembershipSpec extends MultiNodeSpec(NodeMembershipMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
import NodeMembershipMultiJvmSpec._
override def initialParticipants = 3
after {
testConductor.enter("after")
}
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
"A set of connected cluster systems" must {
"(when two nodes) start gossiping to each other so that both nodes gets the same gossip info" taggedAs LongRunningTest in {
// make sure that the node-to-join is started before other join
runOn(first) {
cluster.self
startClusterNode()
}
testConductor.enter("first-started")
enterBarrier("first-started")
runOn(first, second) {
cluster.join(firstAddress)
cluster.join(first)
awaitCond(cluster.latestGossip.members.size == 2)
assertMembers(cluster.latestGossip.members, firstAddress, secondAddress)
assertMembers(cluster.latestGossip.members, first, second)
awaitCond {
cluster.latestGossip.members.forall(_.status == MemberStatus.Up)
}
awaitCond(cluster.convergence.isDefined)
}
enterBarrier("after-1")
}
"(when three nodes) start gossiping to each other so that all nodes gets the same gossip info" taggedAs LongRunningTest in {
runOn(third) {
cluster.join(firstAddress)
cluster.join(first)
}
awaitCond(cluster.latestGossip.members.size == 3)
assertMembers(cluster.latestGossip.members, firstAddress, secondAddress, thirdAddress)
assertMembers(cluster.latestGossip.members, first, second, third)
awaitCond {
cluster.latestGossip.members.forall(_.status == MemberStatus.Up)
}
awaitCond(cluster.convergence.isDefined)
enterBarrier("after-2")
}
}
}

View file

@ -1,74 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
object NodeStartupMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeStartupMultiJvmNode1 extends NodeStartupSpec
class NodeStartupMultiJvmNode2 extends NodeStartupSpec
abstract class NodeStartupSpec extends MultiNodeSpec(NodeStartupMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
import NodeStartupMultiJvmSpec._
override def initialParticipants = 2
after {
testConductor.enter("after")
}
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
"A first cluster node with a 'node-to-join' config set to empty string (singleton cluster)" must {
"be a singleton cluster when started up" taggedAs LongRunningTest in {
runOn(first) {
awaitCond(cluster.isSingletonCluster)
// FIXME #2117 singletonCluster should reach convergence
//awaitCond(cluster.convergence.isDefined)
}
}
"be in 'Joining' phase when started up" taggedAs LongRunningTest in {
runOn(first) {
val members = cluster.latestGossip.members
members.size must be(1)
val joiningMember = members find (_.address == firstAddress)
joiningMember must not be (None)
joiningMember.get.status must be(MemberStatus.Joining)
}
}
}
"A second cluster node" must {
"join the other node cluster when sending a Join command" taggedAs LongRunningTest in {
runOn(second) {
cluster.join(firstAddress)
}
awaitCond {
cluster.latestGossip.members.exists { member
member.address == secondAddress && member.status == MemberStatus.Up
}
}
cluster.latestGossip.members.size must be(2)
awaitCond(cluster.convergence.isDefined)
}
}
}

View file

@ -0,0 +1,65 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
import scala.collection.immutable.SortedSet
import java.util.concurrent.atomic.AtomicReference
object NodeUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeUpMultiJvmNode1 extends NodeUpSpec with FailureDetectorPuppetStrategy
class NodeUpMultiJvmNode2 extends NodeUpSpec with FailureDetectorPuppetStrategy
abstract class NodeUpSpec
extends MultiNodeSpec(NodeUpMultiJvmSpec)
with MultiNodeClusterSpec {
import NodeUpMultiJvmSpec._
"A cluster node that is joining another cluster" must {
"be moved to UP by the leader after a convergence" taggedAs LongRunningTest in {
awaitClusterUp(first, second)
enterBarrier("after-1")
}
"be unaffected when joining again" taggedAs LongRunningTest in {
val unexpected = new AtomicReference[SortedSet[Member]]
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
if (members.size != 2 || members.exists(_.status != MemberStatus.Up))
unexpected.set(members)
}
})
enterBarrier("listener-registered")
runOn(second) {
cluster.join(first)
}
enterBarrier("joined-again")
// let it run for a while to make sure that nothing bad happens
for (n 1 to 20) {
100.millis.dilated.sleep()
unexpected.get must be(null)
cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true)
}
enterBarrier("after-2")
}
}
}

View file

@ -0,0 +1,64 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
object SingletonClusterMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("""
akka.cluster {
auto-down = on
failure-detector.threshold = 4
}
""")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class SingletonClusterWithFailureDetectorPuppetMultiJvmNode1 extends SingletonClusterSpec with FailureDetectorPuppetStrategy
class SingletonClusterWithFailureDetectorPuppetMultiJvmNode2 extends SingletonClusterSpec with FailureDetectorPuppetStrategy
class SingletonClusterWithAccrualFailureDetectorMultiJvmNode1 extends SingletonClusterSpec with AccrualFailureDetectorStrategy
class SingletonClusterWithAccrualFailureDetectorMultiJvmNode2 extends SingletonClusterSpec with AccrualFailureDetectorStrategy
abstract class SingletonClusterSpec
extends MultiNodeSpec(SingletonClusterMultiJvmSpec)
with MultiNodeClusterSpec {
import SingletonClusterMultiJvmSpec._
"A cluster of 2 nodes" must {
"not be singleton cluster when joined" taggedAs LongRunningTest in {
awaitClusterUp(first, second)
cluster.isSingletonCluster must be(false)
assertLeader(first, second)
enterBarrier("after-1")
}
"become singleton cluster when one node is shutdown" taggedAs LongRunningTest in {
runOn(first) {
val secondAddress = address(second)
testConductor.shutdown(second, 0)
markNodeAsUnavailable(secondAddress)
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
cluster.isSingletonCluster must be(true)
assertLeader(first)
}
enterBarrier("after-2")
}
}
}

View file

@ -0,0 +1,77 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
import java.util.concurrent.atomic.AtomicReference
import scala.collection.immutable.SortedSet
object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
val fifth = role("fifth")
// Note that this test uses default configuration,
// not MultiNodeClusterSpec.clusterConfig
commonConfig(ConfigFactory.parseString("""
akka.cluster {
nr-of-deputy-nodes = 0
}
akka.loglevel = INFO
"""))
}
class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
abstract class SunnyWeatherSpec
extends MultiNodeSpec(SunnyWeatherMultiJvmSpec)
with MultiNodeClusterSpec {
import SunnyWeatherMultiJvmSpec._
"A normal cluster" must {
"be healthy" taggedAs LongRunningTest in {
// start some
awaitClusterUp(first, second, third)
runOn(first, second, third) {
log.info("3 joined")
}
// add a few more
awaitClusterUp(roles: _*)
log.info("5 joined")
val unexpected = new AtomicReference[SortedSet[Member]]
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
// we don't expected any changes to the cluster
unexpected.set(members)
}
})
for (n 1 to 30) {
enterBarrier("period-" + n)
unexpected.get must be(null)
awaitUpConvergence(roles.size)
assertLeaderIn(roles)
if (n % 5 == 0) log.info("Passed period [{}]", n)
1.seconds.sleep
}
enterBarrier("after")
}
}
}

View file

@ -0,0 +1,400 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.actor.Address
import akka.remote.testconductor.RoleName
import MemberStatus._
object TransitionMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
val fifth = role("fifth")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("akka.cluster.periodic-tasks-initial-delay = 300 s # turn off all periodic tasks")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class TransitionMultiJvmNode1 extends TransitionSpec with FailureDetectorPuppetStrategy
class TransitionMultiJvmNode2 extends TransitionSpec with FailureDetectorPuppetStrategy
class TransitionMultiJvmNode3 extends TransitionSpec with FailureDetectorPuppetStrategy
class TransitionMultiJvmNode4 extends TransitionSpec with FailureDetectorPuppetStrategy
class TransitionMultiJvmNode5 extends TransitionSpec with FailureDetectorPuppetStrategy
abstract class TransitionSpec
extends MultiNodeSpec(TransitionMultiJvmSpec)
with MultiNodeClusterSpec {
import TransitionMultiJvmSpec._
// sorted in the order used by the cluster
def leader(roles: RoleName*) = roles.sorted.head
def nonLeader(roles: RoleName*) = roles.toSeq.sorted.tail
def memberStatus(address: Address): MemberStatus = {
val statusOption = (cluster.latestGossip.members ++ cluster.latestGossip.overview.unreachable).collectFirst {
case m if m.address == address m.status
}
statusOption must not be (None)
statusOption.get
}
def memberAddresses: Set[Address] = cluster.latestGossip.members.map(_.address)
def members: Set[RoleName] = memberAddresses.flatMap(roleName(_))
def seenLatestGossip: Set[RoleName] = {
val gossip = cluster.latestGossip
gossip.overview.seen.collect {
case (address, v) if v == gossip.version roleName(address)
}.flatten.toSet
}
def awaitSeen(addresses: Address*): Unit = awaitCond {
(seenLatestGossip map address) == addresses.toSet
}
def awaitMembers(addresses: Address*): Unit = awaitCond {
memberAddresses == addresses.toSet
}
def awaitMemberStatus(address: Address, status: MemberStatus): Unit = awaitCond {
memberStatus(address) == status
}
// DSL sugar for `role1 gossipTo role2`
implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role)
var gossipBarrierCounter = 0
class RoleWrapper(fromRole: RoleName) {
def gossipTo(toRole: RoleName): Unit = {
gossipBarrierCounter += 1
runOn(toRole) {
val g = cluster.latestGossip
enterBarrier("before-gossip-" + gossipBarrierCounter)
awaitCond(cluster.latestGossip != g) // received gossip
enterBarrier("after-gossip-" + gossipBarrierCounter)
}
runOn(fromRole) {
enterBarrier("before-gossip-" + gossipBarrierCounter)
cluster.gossipTo(toRole) // send gossip
enterBarrier("after-gossip-" + gossipBarrierCounter)
}
runOn(roles.filterNot(r r == fromRole || r == toRole): _*) {
enterBarrier("before-gossip-" + gossipBarrierCounter)
enterBarrier("after-gossip-" + gossipBarrierCounter)
}
}
}
"A Cluster" must {
"start nodes as singleton clusters" taggedAs LongRunningTest in {
startClusterNode()
cluster.isSingletonCluster must be(true)
cluster.status must be(Joining)
cluster.convergence.isDefined must be(true)
cluster.leaderActions()
cluster.status must be(Up)
enterBarrier("after-1")
}
"perform correct transitions when second joining first" taggedAs LongRunningTest in {
runOn(second) {
cluster.join(first)
}
runOn(first) {
awaitMembers(first, second)
memberStatus(first) must be(Up)
memberStatus(second) must be(Joining)
seenLatestGossip must be(Set(first))
cluster.convergence.isDefined must be(false)
}
enterBarrier("second-joined")
first gossipTo second
second gossipTo first
runOn(first, second) {
memberStatus(first) must be(Up)
memberStatus(second) must be(Joining)
seenLatestGossip must be(Set(first, second))
cluster.convergence.isDefined must be(true)
}
enterBarrier("convergence-joining-2")
runOn(leader(first, second)) {
cluster.leaderActions()
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
}
enterBarrier("leader-actions-2")
leader(first, second) gossipTo nonLeader(first, second).head
runOn(nonLeader(first, second).head) {
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
seenLatestGossip must be(Set(first, second))
cluster.convergence.isDefined must be(true)
}
nonLeader(first, second).head gossipTo leader(first, second)
runOn(first, second) {
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
seenLatestGossip must be(Set(first, second))
cluster.convergence.isDefined must be(true)
}
enterBarrier("after-2")
}
"perform correct transitions when third joins second" taggedAs LongRunningTest in {
runOn(third) {
cluster.join(second)
}
runOn(second) {
awaitMembers(first, second, third)
cluster.convergence.isDefined must be(false)
memberStatus(third) must be(Joining)
seenLatestGossip must be(Set(second))
}
enterBarrier("third-joined-second")
second gossipTo first
runOn(first) {
members must be(Set(first, second, third))
memberStatus(third) must be(Joining)
seenLatestGossip must be(Set(first, second))
cluster.convergence.isDefined must be(false)
}
first gossipTo third
third gossipTo first
third gossipTo second
runOn(first, second, third) {
members must be(Set(first, second, third))
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
memberStatus(third) must be(Joining)
seenLatestGossip must be(Set(first, second, third))
cluster.convergence.isDefined must be(true)
}
enterBarrier("convergence-joining-3")
runOn(leader(first, second, third)) {
cluster.leaderActions()
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
memberStatus(third) must be(Up)
}
enterBarrier("leader-actions-3")
// leader gossipTo first non-leader
leader(first, second, third) gossipTo nonLeader(first, second, third).head
runOn(nonLeader(first, second, third).head) {
memberStatus(third) must be(Up)
seenLatestGossip must be(Set(leader(first, second, third), myself))
cluster.convergence.isDefined must be(false)
}
// first non-leader gossipTo the other non-leader
nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head
runOn(nonLeader(first, second, third).head) {
cluster.gossipTo(nonLeader(first, second, third).tail.head)
}
runOn(nonLeader(first, second, third).tail.head) {
memberStatus(third) must be(Up)
seenLatestGossip must be(Set(first, second, third))
cluster.convergence.isDefined must be(true)
}
// and back again
nonLeader(first, second, third).tail.head gossipTo nonLeader(first, second, third).head
runOn(nonLeader(first, second, third).head) {
memberStatus(third) must be(Up)
seenLatestGossip must be(Set(first, second, third))
cluster.convergence.isDefined must be(true)
}
// first non-leader gossipTo the leader
nonLeader(first, second, third).head gossipTo leader(first, second, third)
runOn(first, second, third) {
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
memberStatus(third) must be(Up)
seenLatestGossip must be(Set(first, second, third))
cluster.convergence.isDefined must be(true)
}
enterBarrier("after-3")
}
"startup a second separated cluster consisting of nodes fourth and fifth" taggedAs LongRunningTest in {
runOn(fourth) {
cluster.join(fifth)
}
runOn(fifth) {
awaitMembers(fourth, fifth)
}
testConductor.enter("fourth-joined")
fifth gossipTo fourth
fourth gossipTo fifth
runOn(fourth, fifth) {
memberStatus(fourth) must be(Joining)
memberStatus(fifth) must be(Up)
seenLatestGossip must be(Set(fourth, fifth))
cluster.convergence.isDefined must be(true)
}
enterBarrier("after-4")
}
"perform correct transitions when second cluster (node fourth) joins first cluster (node third)" taggedAs LongRunningTest in {
runOn(fourth) {
cluster.join(third)
}
runOn(third) {
awaitMembers(first, second, third, fourth)
seenLatestGossip must be(Set(third))
}
enterBarrier("fourth-joined-third")
third gossipTo second
runOn(second) {
seenLatestGossip must be(Set(second, third))
}
second gossipTo fourth
runOn(fourth) {
members must be(roles.toSet)
// merge conflict
seenLatestGossip must be(Set(fourth))
}
fourth gossipTo first
fourth gossipTo second
fourth gossipTo third
fourth gossipTo fifth
runOn(first, second, third, fifth) {
members must be(roles.toSet)
seenLatestGossip must be(Set(fourth, myself))
}
first gossipTo fifth
runOn(fifth) {
seenLatestGossip must be(Set(first, fourth, fifth))
}
fifth gossipTo third
runOn(third) {
seenLatestGossip must be(Set(first, third, fourth, fifth))
}
third gossipTo second
runOn(second) {
seenLatestGossip must be(roles.toSet)
cluster.convergence.isDefined must be(true)
}
second gossipTo first
second gossipTo third
second gossipTo fourth
third gossipTo fifth
seenLatestGossip must be(roles.toSet)
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
memberStatus(third) must be(Up)
memberStatus(fourth) must be(Joining)
memberStatus(fifth) must be(Up)
cluster.convergence.isDefined must be(true)
enterBarrier("convergence-joining-3")
runOn(leader(roles: _*)) {
cluster.leaderActions()
memberStatus(fourth) must be(Up)
seenLatestGossip must be(Set(myself))
cluster.convergence.isDefined must be(false)
}
// spread the word
for (x :: y :: Nil (roles.sorted ++ roles.sorted.dropRight(1)).toList.sliding(2)) {
x gossipTo y
}
enterBarrier("spread-5")
seenLatestGossip must be(roles.toSet)
memberStatus(first) must be(Up)
memberStatus(second) must be(Up)
memberStatus(third) must be(Up)
memberStatus(fourth) must be(Up)
memberStatus(fifth) must be(Up)
cluster.convergence.isDefined must be(true)
enterBarrier("after-5")
}
"perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in {
runOn(fifth) {
markNodeAsUnavailable(second)
cluster.reapUnreachableMembers()
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
seenLatestGossip must be(Set(fifth))
}
enterBarrier("after-second-unavailble")
// spread the word
val gossipRound = List(fifth, fourth, third, first, third, fourth, fifth)
for (x :: y :: Nil gossipRound.sliding(2)) {
x gossipTo y
}
runOn((roles.filterNot(_ == second)): _*) {
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
cluster.convergence.isDefined must be(false)
}
runOn(third) {
cluster.down(second)
awaitMemberStatus(second, Down)
}
enterBarrier("after-second-down")
// spread the word
val gossipRound2 = List(third, fourth, fifth, first, third, fourth, fifth)
for (x :: y :: Nil gossipRound2.sliding(2)) {
x gossipTo y
}
runOn((roles.filterNot(_ == second)): _*) {
cluster.latestGossip.overview.unreachable must contain(Member(second, Down))
memberStatus(second) must be(Down)
seenLatestGossip must be(Set(first, third, fourth, fifth))
cluster.convergence.isDefined must be(true)
}
enterBarrier("after-6")
}
}
}

View file

@ -6,6 +6,9 @@ package akka.cluster
import akka.actor.Address
import akka.testkit.{ LongRunningTest, AkkaSpec }
import scala.collection.immutable.TreeMap
import akka.util.duration._
import akka.util.Duration
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class AccrualFailureDetectorSpec extends AkkaSpec("""
@ -17,7 +20,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
val conn = Address("akka", "", "localhost", 2552)
val conn2 = Address("akka", "", "localhost", 2553)
def fakeTimeGenerator(timeIntervals: List[Long]): () Long = {
def fakeTimeGenerator(timeIntervals: Seq[Long]): () Long = {
var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) acc ::: List[Long](acc.last + c))
def timeGenerator(): Long = {
val currentTime = times.head
@ -27,22 +30,85 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
timeGenerator
}
"return phi value of 0.0D on startup for each address" in {
val fd = new AccrualFailureDetector(system, conn)
fd.phi(conn) must be(0.0D)
fd.phi(conn2) must be(0.0D)
val defaultFakeTimeIntervals = Vector.fill(20)(1000L)
def createFailureDetector(
threshold: Double = 8.0,
maxSampleSize: Int = 1000,
minStdDeviation: Duration = 10.millis,
acceptableLostDuration: Duration = Duration.Zero,
firstHeartbeatEstimate: Duration = 1.second,
clock: () Long = fakeTimeGenerator(defaultFakeTimeIntervals)): AccrualFailureDetector =
new AccrualFailureDetector(system,
threshold,
maxSampleSize,
minStdDeviation,
acceptableLostDuration,
firstHeartbeatEstimate = firstHeartbeatEstimate,
clock = clock)
"use good enough cumulative distribution function" in {
val fd = createFailureDetector()
fd.cumulativeDistributionFunction(0.0, 0, 1) must be(0.5 plusOrMinus (0.001))
fd.cumulativeDistributionFunction(0.6, 0, 1) must be(0.7257 plusOrMinus (0.001))
fd.cumulativeDistributionFunction(1.5, 0, 1) must be(0.9332 plusOrMinus (0.001))
fd.cumulativeDistributionFunction(2.0, 0, 1) must be(0.97725 plusOrMinus (0.01))
fd.cumulativeDistributionFunction(2.5, 0, 1) must be(0.9379 plusOrMinus (0.1))
fd.cumulativeDistributionFunction(3.5, 0, 1) must be(0.99977 plusOrMinus (0.1))
fd.cumulativeDistributionFunction(4.0, 0, 1) must be(0.99997 plusOrMinus (0.1))
for (x :: y :: Nil (0.0 to 4.0 by 0.1).toList.sliding(2)) {
fd.cumulativeDistributionFunction(x, 0, 1) must be < (
fd.cumulativeDistributionFunction(y, 0, 1))
}
fd.cumulativeDistributionFunction(2.2, 2.0, 0.3) must be(0.7475 plusOrMinus (0.001))
}
"return realistic phi values" in {
val fd = createFailureDetector()
val test = TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3)
for ((timeDiff, expectedPhi) test) {
fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1))
}
// larger stdDeviation results => lower phi
fd.phi(timeDiff = 1100, mean = 1000.0, stdDeviation = 500.0) must be < (
fd.phi(timeDiff = 1100, mean = 1000.0, stdDeviation = 100.0))
}
"return phi value of 0.0 on startup for each address, when no heartbeats" in {
val fd = createFailureDetector()
fd.phi(conn) must be(0.0)
fd.phi(conn2) must be(0.0)
}
"return phi based on guess when only one heartbeat" in {
val timeInterval = List[Long](0, 1000, 1000, 1000, 1000)
val fd = createFailureDetector(firstHeartbeatEstimate = 1.seconds,
clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.phi(conn) must be(0.3 plusOrMinus 0.2)
fd.phi(conn) must be(4.5 plusOrMinus 0.3)
fd.phi(conn) must be > (15.0)
}
"return phi value using first interval after second heartbeat" in {
val timeInterval = List[Long](0, 100, 100, 100)
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.phi(conn) must be > (0.0)
fd.heartbeat(conn)
fd.phi(conn) must be > (0.0)
}
"mark node as available after a series of successful heartbeats" in {
val timeInterval = List[Long](0, 1000, 100, 100)
val ft = fakeTimeGenerator(timeInterval)
val fd = new AccrualFailureDetector(system, conn, timeMachine = ft)
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
@ -50,18 +116,12 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
"mark node as dead after explicit removal of connection" in {
val timeInterval = List[Long](0, 1000, 100, 100, 100)
val ft = fakeTimeGenerator(timeInterval)
val fd = new AccrualFailureDetector(system, conn, timeMachine = ft)
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
fd.remove(conn)
fd.isAvailable(conn) must be(false)
@ -69,14 +129,11 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
"mark node as available after explicit removal of connection and receiving heartbeat again" in {
val timeInterval = List[Long](0, 1000, 100, 1100, 1100, 1100, 1100, 1100, 100)
val ft = fakeTimeGenerator(timeInterval)
val fd = new AccrualFailureDetector(system, conn, timeMachine = ft)
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn) //0
fd.heartbeat(conn) //1000
fd.heartbeat(conn) //1100
fd.isAvailable(conn) must be(true) //2200
@ -87,60 +144,72 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
// it receives heartbeat from an explicitly removed node
fd.heartbeat(conn) //4400
fd.heartbeat(conn) //5500
fd.heartbeat(conn) //6600
fd.isAvailable(conn) must be(true) //6700
}
"mark node as dead if heartbeat are missed" in {
val timeInterval = List[Long](0, 1000, 100, 100, 5000)
val timeInterval = List[Long](0, 1000, 100, 100, 7000)
val ft = fakeTimeGenerator(timeInterval)
val fd = new AccrualFailureDetector(system, conn, threshold = 3, timeMachine = ft)
val fd = createFailureDetector(threshold = 3, clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn) //0
fd.heartbeat(conn) //1000
fd.heartbeat(conn) //1100
fd.isAvailable(conn) must be(true) //1200
fd.isAvailable(conn) must be(false) //6200
fd.isAvailable(conn) must be(false) //8200
}
"mark node as available if it starts heartbeat again after being marked dead due to detection of failure" in {
val timeInterval = List[Long](0, 1000, 100, 1100, 5000, 100, 1000, 100, 100)
val ft = fakeTimeGenerator(timeInterval)
val fd = new AccrualFailureDetector(system, conn, threshold = 3, timeMachine = ft)
val timeInterval = List[Long](0, 1000, 100, 1100, 7000, 100, 1000, 100, 100)
val fd = createFailureDetector(threshold = 3, clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn) //0
fd.heartbeat(conn) //1000
fd.heartbeat(conn) //1100
fd.isAvailable(conn) must be(true) //1200
fd.isAvailable(conn) must be(false) //8200
fd.heartbeat(conn) //8300
fd.heartbeat(conn) //9300
fd.heartbeat(conn) //9400
fd.isAvailable(conn) must be(false) //6200
fd.isAvailable(conn) must be(true) //9500
}
fd.heartbeat(conn) //6300
"accept some configured missing heartbeats" in {
val timeInterval = List[Long](0, 1000, 1000, 1000, 4000, 1000, 1000)
val fd = createFailureDetector(acceptableLostDuration = 3.seconds, clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn) //7300
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
}
fd.heartbeat(conn) //7400
"fail after configured acceptable missing heartbeats" in {
val timeInterval = List[Long](0, 1000, 1000, 1000, 1000, 1000, 500, 500, 5000)
val fd = createFailureDetector(acceptableLostDuration = 3.seconds, clock = fakeTimeGenerator(timeInterval))
fd.isAvailable(conn) must be(true) //7500
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(false)
}
"use maxSampleSize heartbeats" in {
val timeInterval = List[Long](0, 100, 100, 100, 100, 600, 1000, 1000, 1000, 1000, 1000)
val ft = fakeTimeGenerator(timeInterval)
val fd = new AccrualFailureDetector(system, conn, maxSampleSize = 3, timeMachine = ft)
val fd = createFailureDetector(maxSampleSize = 3, clock = fakeTimeGenerator(timeInterval))
// 100 ms interval
fd.heartbeat(conn) //0
@ -156,5 +225,35 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
val phi2 = fd.phi(conn) //5000
phi2 must be(phi1.plusOrMinus(0.001))
}
}
"Statistics for heartbeats" must {
"calculate correct mean and variance" in {
val samples = Seq(100, 200, 125, 340, 130)
val stats = (HeartbeatHistory(maxSampleSize = 20) /: samples) { (stats, value) stats :+ value }
stats.mean must be(179.0 plusOrMinus 0.00001)
stats.variance must be(7584.0 plusOrMinus 0.00001)
}
"have 0.0 variance for one sample" in {
(HeartbeatHistory(600) :+ 1000L).variance must be(0.0 plusOrMinus 0.00001)
}
"be capped by the specified maxSampleSize" in {
val history3 = HeartbeatHistory(maxSampleSize = 3) :+ 100 :+ 110 :+ 90
history3.mean must be(100.0 plusOrMinus 0.00001)
history3.variance must be(66.6666667 plusOrMinus 0.00001)
val history4 = history3 :+ 140
history4.mean must be(113.333333 plusOrMinus 0.00001)
history4.variance must be(422.222222 plusOrMinus 0.00001)
val history5 = history4 :+ 80
history5.mean must be(103.333333 plusOrMinus 0.00001)
history5.variance must be(688.88888889 plusOrMinus 0.00001)
}
}
}

View file

@ -16,16 +16,23 @@ class ClusterConfigSpec extends AkkaSpec {
"be able to parse generic cluster config elements" in {
val settings = new ClusterSettings(system.settings.config, system.name)
import settings._
FailureDetectorThreshold must be(8)
FailureDetectorThreshold must be(8.0 plusOrMinus 0.0001)
FailureDetectorMaxSampleSize must be(1000)
FailureDetectorImplementationClass must be(classOf[AccrualFailureDetector].getName)
FailureDetectorMinStdDeviation must be(100 millis)
FailureDetectorAcceptableHeartbeatPause must be(3 seconds)
NodeToJoin must be(None)
PeriodicTasksInitialDelay must be(1 seconds)
GossipFrequency must be(1 second)
LeaderActionsFrequency must be(1 second)
UnreachableNodesReaperFrequency must be(1 second)
GossipInterval must be(1 second)
HeartbeatInterval must be(1 second)
LeaderActionsInterval must be(1 second)
UnreachableNodesReaperInterval must be(1 second)
JoinTimeout must be(60 seconds)
NrOfGossipDaemons must be(4)
NrOfDeputyNodes must be(3)
AutoDown must be(true)
SchedulerTickDuration must be(33 millis)
SchedulerTicksPerWheel must be(512)
}
}
}

View file

@ -0,0 +1,217 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.testkit.AkkaSpec
import akka.util.duration._
import akka.util.Duration
import akka.actor.ExtendedActorSystem
import akka.actor.Address
import java.util.concurrent.atomic.AtomicInteger
import org.scalatest.BeforeAndAfter
object ClusterSpec {
val config = """
akka.cluster {
auto-down = off
nr-of-deputy-nodes = 3
periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks
}
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
akka.remote.netty.port = 0
# akka.loglevel = DEBUG
"""
case class GossipTo(address: Address)
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
import ClusterSpec._
val deterministicRandom = new AtomicInteger
val failureDetector = new FailureDetectorPuppet(system)
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) {
override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = {
if (addresses.isEmpty) None
else Some(addresses.toSeq(deterministicRandom.getAndIncrement % addresses.size))
}
override def gossipTo(address: Address): Unit = {
if (address == self.address) {
super.gossipTo(address)
}
// represent the gossip with a message to be used in asserts
testActor ! GossipTo(address)
}
@volatile
var _gossipToUnreachableProbablity = 0.0
override def gossipToUnreachableProbablity(membersSize: Int, unreachableSize: Int): Double = {
if (_gossipToUnreachableProbablity < 0.0) super.gossipToUnreachableProbablity(membersSize, unreachableSize)
else _gossipToUnreachableProbablity
}
@volatile
var _gossipToDeputyProbablity = 0.0
override def gossipToDeputyProbablity(membersSize: Int, unreachableSize: Int, deputySize: Int): Double = {
if (_gossipToDeputyProbablity < 0.0) super.gossipToDeputyProbablity(membersSize, unreachableSize, deputySize)
else _gossipToDeputyProbablity
}
}
val selfAddress = cluster.self.address
val addresses = IndexedSeq(
selfAddress,
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 1),
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 2),
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 3),
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 4),
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 5))
def memberStatus(address: Address): Option[MemberStatus] =
cluster.latestGossip.members.collectFirst { case m if m.address == address m.status }
before {
cluster._gossipToUnreachableProbablity = 0.0
cluster._gossipToDeputyProbablity = 0.0
addresses foreach failureDetector.remove
deterministicRandom.set(0)
}
"A Cluster" must {
"initially be singleton cluster and reach convergence immediately" in {
cluster.isSingletonCluster must be(true)
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress))
memberStatus(selfAddress) must be(Some(MemberStatus.Joining))
cluster.convergence.isDefined must be(true)
cluster.leaderActions()
memberStatus(selfAddress) must be(Some(MemberStatus.Up))
}
"accept a joining node" in {
cluster.joining(addresses(1))
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress, addresses(1)))
memberStatus(addresses(1)) must be(Some(MemberStatus.Joining))
cluster.convergence.isDefined must be(false)
}
"accept a few more joining nodes" in {
for (a addresses.drop(2)) {
cluster.joining(a)
memberStatus(a) must be(Some(MemberStatus.Joining))
}
cluster.latestGossip.members.map(_.address) must be(addresses.toSet)
}
"order members by host and port" in {
// note the importance of using toSeq before map, otherwise it will not preserve the order
cluster.latestGossip.members.toSeq.map(_.address) must be(addresses.toSeq)
}
"gossip to random live node" in {
cluster.latestGossip.members
cluster.gossip()
cluster.gossip()
cluster.gossip()
cluster.gossip()
expectMsg(GossipTo(addresses(1)))
expectMsg(GossipTo(addresses(2)))
expectMsg(GossipTo(addresses(3)))
expectMsg(GossipTo(addresses(4)))
expectNoMsg(1 second)
}
"use certain probability for gossiping to unreachable node depending on the number of unreachable and live nodes" in {
cluster._gossipToUnreachableProbablity = -1.0 // use real impl
cluster.gossipToUnreachableProbablity(10, 1) must be < (cluster.gossipToUnreachableProbablity(9, 1))
cluster.gossipToUnreachableProbablity(10, 1) must be < (cluster.gossipToUnreachableProbablity(10, 2))
cluster.gossipToUnreachableProbablity(10, 5) must be < (cluster.gossipToUnreachableProbablity(10, 9))
cluster.gossipToUnreachableProbablity(0, 10) must be <= (1.0)
cluster.gossipToUnreachableProbablity(1, 10) must be <= (1.0)
cluster.gossipToUnreachableProbablity(10, 0) must be(0.0 plusOrMinus (0.0001))
cluster.gossipToUnreachableProbablity(0, 0) must be(0.0 plusOrMinus (0.0001))
}
"use certain probability for gossiping to deputy node depending on the number of unreachable and live nodes" in {
cluster._gossipToDeputyProbablity = -1.0 // use real impl
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(9, 1, 2))
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 2))
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 3))
cluster.gossipToDeputyProbablity(10, 5, 5) must be < (cluster.gossipToDeputyProbablity(10, 9, 5))
cluster.gossipToDeputyProbablity(0, 10, 0) must be <= (1.0)
cluster.gossipToDeputyProbablity(1, 10, 1) must be <= (1.0)
cluster.gossipToDeputyProbablity(10, 0, 0) must be(0.0 plusOrMinus (0.0001))
cluster.gossipToDeputyProbablity(0, 0, 0) must be(0.0 plusOrMinus (0.0001))
cluster.gossipToDeputyProbablity(4, 0, 4) must be(1.0 plusOrMinus (0.0001))
cluster.gossipToDeputyProbablity(3, 7, 4) must be(1.0 plusOrMinus (0.0001))
}
"gossip to duputy node" in {
cluster._gossipToDeputyProbablity = 1.0 // always
// we have configured 2 deputy nodes
cluster.gossip() // 1 is deputy
cluster.gossip() // 2 is deputy
cluster.gossip() // 3 is deputy
cluster.gossip() // 4 is not deputy, and therefore a deputy is also used
expectMsg(GossipTo(addresses(1)))
expectMsg(GossipTo(addresses(2)))
expectMsg(GossipTo(addresses(3)))
expectMsg(GossipTo(addresses(4)))
// and the extra gossip to deputy
expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3)))
expectNoMsg(1 second)
}
"gossip to random unreachable node" in {
val dead = Set(addresses(1))
dead foreach failureDetector.markNodeAsUnavailable
cluster._gossipToUnreachableProbablity = 1.0 // always
cluster.reapUnreachableMembers()
cluster.latestGossip.overview.unreachable.map(_.address) must be(dead)
cluster.gossip()
expectMsg(GossipTo(addresses(2))) // first available
expectMsg(GossipTo(addresses(1))) // the unavailable
expectNoMsg(1 second)
}
"gossip to random deputy node if number of live nodes is less than number of deputy nodes" in {
cluster._gossipToDeputyProbablity = -1.0 // real impl
// 0 and 2 still alive
val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5))
dead foreach failureDetector.markNodeAsUnavailable
cluster.reapUnreachableMembers()
cluster.latestGossip.overview.unreachable.map(_.address) must be(dead)
for (n 1 to 20) {
cluster.gossip()
expectMsg(GossipTo(addresses(2))) // the only available
// and always to one of the 3 deputies
expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3)))
}
expectNoMsg(1 second)
}
}
}

View file

@ -0,0 +1,60 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor.{ Address, ActorSystem }
import akka.event.{ Logging, LogSource }
/**
* User controllable "puppet" failure detector.
*/
class FailureDetectorPuppet(system: ActorSystem, settings: ClusterSettings) extends FailureDetector {
import java.util.concurrent.ConcurrentHashMap
def this(system: ActorSystem) = this(system, new ClusterSettings(system.settings.config, system.name))
trait Status
object Up extends Status
object Down extends Status
implicit private val logSource: LogSource[AnyRef] = new LogSource[AnyRef] {
def genString(o: AnyRef): String = o.getClass.getName
override def getClazz(o: AnyRef): Class[_] = o.getClass
}
private val log = Logging(system, this)
private val connections = new ConcurrentHashMap[Address, Status]
def markNodeAsUnavailable(connection: Address): this.type = {
connections.put(connection, Down)
this
}
def markNodeAsAvailable(connection: Address): this.type = {
connections.put(connection, Up)
this
}
def isAvailable(connection: Address): Boolean = connections.get(connection) match {
case null
log.debug("Adding cluster node [{}]", connection)
connections.put(connection, Up)
true
case Up
log.debug("isAvailable: Cluster node IS NOT available [{}]", connection)
true
case Down
log.debug("isAvailable: Cluster node IS available [{}]", connection)
false
}
def heartbeat(connection: Address): Unit = log.debug("Heart beat from cluster node[{}]", connection)
def remove(connection: Address): Unit = {
log.debug("Removing cluster node [{}]", connection)
connections.remove(connection)
}
}

View file

@ -0,0 +1,43 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.testkit.AkkaSpec
import akka.util.duration._
import akka.testkit.TimingTest
import akka.testkit.TestLatch
import akka.dispatch.Await
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class FixedRateTaskSpec extends AkkaSpec {
"Task scheduled at fixed rate" must {
"adjust for scheduler inaccuracy" taggedAs TimingTest in {
val startTime = System.nanoTime
val n = 33
val latch = new TestLatch(n)
FixedRateTask(system.scheduler, 150.millis, 150.millis) {
latch.countDown()
}
Await.ready(latch, 6.seconds)
val rate = n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis
rate must be(6.66 plusOrMinus (0.4))
}
"compensate for long running task" taggedAs TimingTest in {
val startTime = System.nanoTime
val n = 22
val latch = new TestLatch(n)
FixedRateTask(system.scheduler, 225.millis, 225.millis) {
80.millis.sleep()
latch.countDown()
}
Await.ready(latch, 6.seconds)
val rate = n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis
rate must be(4.4 plusOrMinus (0.3))
}
}
}

View file

@ -0,0 +1,104 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.actor.Address
import scala.collection.immutable.SortedSet
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class GossipSpec extends WordSpec with MustMatchers {
import MemberStatus._
val a1 = Member(Address("akka", "sys", "a", 2552), Up)
val a2 = Member(Address("akka", "sys", "a", 2552), Joining)
val b1 = Member(Address("akka", "sys", "b", 2552), Up)
val b2 = Member(Address("akka", "sys", "b", 2552), Removed)
val c1 = Member(Address("akka", "sys", "c", 2552), Leaving)
val c2 = Member(Address("akka", "sys", "c", 2552), Up)
val d1 = Member(Address("akka", "sys", "d", 2552), Leaving)
val d2 = Member(Address("akka", "sys", "d", 2552), Removed)
val e1 = Member(Address("akka", "sys", "e", 2552), Joining)
val e2 = Member(Address("akka", "sys", "e", 2552), Up)
"A Gossip" must {
"merge members by status priority" in {
val g1 = Gossip(members = SortedSet(a1, c1, e1))
val g2 = Gossip(members = SortedSet(a2, c2, e2))
val merged1 = g1 merge g2
merged1.members must be(SortedSet(a2, c1, e1))
merged1.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining))
val merged2 = g2 merge g1
merged2.members must be(SortedSet(a2, c1, e1))
merged2.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining))
}
"merge unreachable by status priority" in {
val g1 = Gossip(members = Gossip.emptyMembers, overview = GossipOverview(unreachable = Set(a1, b1, c1, d1)))
val g2 = Gossip(members = Gossip.emptyMembers, overview = GossipOverview(unreachable = Set(a2, b2, c2, d2)))
val merged1 = g1 merge g2
merged1.overview.unreachable must be(Set(a2, b2, c1, d2))
merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed))
val merged2 = g2 merge g1
merged2.overview.unreachable must be(Set(a2, b2, c1, d2))
merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed))
}
"merge by excluding unreachable from members" in {
val g1 = Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(c1, d1)))
val g2 = Gossip(members = SortedSet(a2, c2), overview = GossipOverview(unreachable = Set(b2, d2)))
val merged1 = g1 merge g2
merged1.members must be(SortedSet(a2))
merged1.members.toSeq.map(_.status) must be(Seq(Joining))
merged1.overview.unreachable must be(Set(b2, c1, d2))
merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Removed, Leaving, Removed))
val merged2 = g2 merge g1
merged2.members must be(SortedSet(a2))
merged2.members.toSeq.map(_.status) must be(Seq(Joining))
merged2.overview.unreachable must be(Set(b2, c1, d2))
merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Removed, Leaving, Removed))
}
"start with fresh seen table after merge" in {
val g1 = Gossip(members = SortedSet(a1, e1)).seen(a1.address).seen(a1.address)
val g2 = Gossip(members = SortedSet(a2, e2)).seen(e2.address).seen(e2.address)
val merged1 = g1 merge g2
merged1.overview.seen.isEmpty must be(true)
val merged2 = g2 merge g1
merged2.overview.seen.isEmpty must be(true)
}
"not have node in both members and unreachable" in intercept[IllegalArgumentException] {
Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(b2)))
}
"not have live members with wrong status" in intercept[IllegalArgumentException] {
// b2 is Removed
Gossip(members = SortedSet(a2, b2))
}
"not have non cluster members in seen table" in intercept[IllegalArgumentException] {
Gossip(members = SortedSet(a1, e1)).seen(a1.address).seen(e1.address).seen(b1.address)
}
}
}

View file

@ -0,0 +1,138 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor.{ Address, AddressFromURIString }
import java.net.InetSocketAddress
import org.scalatest.matchers.MustMatchers
import org.scalatest.WordSpec
import scala.collection.immutable.SortedSet
import scala.util.Random
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MemberOrderingSpec extends WordSpec with MustMatchers {
import Member.ordering
import Member.addressOrdering
import MemberStatus._
"An Ordering[Member]" must {
"order non-exiting members by host:port" in {
val members = SortedSet.empty[Member] +
Member(AddressFromURIString("akka://sys@darkstar:1112"), Up) +
Member(AddressFromURIString("akka://sys@darkstar:1113"), Joining) +
Member(AddressFromURIString("akka://sys@darkstar:1111"), Up)
val seq = members.toSeq
seq.size must equal(3)
seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Up))
seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Up))
seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Joining))
}
"order exiting members by last" in {
val members = SortedSet.empty[Member] +
Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting) +
Member(AddressFromURIString("akka://sys@darkstar:1113"), Up) +
Member(AddressFromURIString("akka://sys@darkstar:1111"), Joining)
val seq = members.toSeq
seq.size must equal(3)
seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Joining))
seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Up))
seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting))
}
"order multiple exiting members by last but internally by host:port" in {
val members = SortedSet.empty[Member] +
Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting) +
Member(AddressFromURIString("akka://sys@darkstar:1113"), Leaving) +
Member(AddressFromURIString("akka://sys@darkstar:1111"), Up) +
Member(AddressFromURIString("akka://sys@darkstar:1110"), Exiting)
val seq = members.toSeq
seq.size must equal(4)
seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Up))
seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Leaving))
seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1110"), Exiting))
seq(3) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting))
}
"be sorted by address correctly" in {
import Member.ordering
// sorting should be done on host and port, only
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
val m2 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
val m3 = Member(Address("cluster", "sys2", "host2", 8000), MemberStatus.Up)
val m4 = Member(Address("cluster", "sys2", "host2", 9000), MemberStatus.Up)
val m5 = Member(Address("cluster", "sys1", "host2", 10000), MemberStatus.Up)
val expected = IndexedSeq(m1, m2, m3, m4, m5)
val shuffled = Random.shuffle(expected)
shuffled.sorted must be(expected)
(SortedSet.empty[Member] ++ shuffled).toIndexedSeq must be(expected)
}
"have stable equals and hashCode" in {
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Joining)
val m2 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
val m3 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
m1 must be(m2)
m1.hashCode must be(m2.hashCode)
m3 must not be (m2)
m3 must not be (m1)
}
}
"An Ordering[Address]" must {
"order addresses by port" in {
val addresses = SortedSet.empty[Address] +
AddressFromURIString("akka://sys@darkstar:1112") +
AddressFromURIString("akka://sys@darkstar:1113") +
AddressFromURIString("akka://sys@darkstar:1110") +
AddressFromURIString("akka://sys@darkstar:1111")
val seq = addresses.toSeq
seq.size must equal(4)
seq(0) must equal(AddressFromURIString("akka://sys@darkstar:1110"))
seq(1) must equal(AddressFromURIString("akka://sys@darkstar:1111"))
seq(2) must equal(AddressFromURIString("akka://sys@darkstar:1112"))
seq(3) must equal(AddressFromURIString("akka://sys@darkstar:1113"))
}
"order addresses by hostname" in {
val addresses = SortedSet.empty[Address] +
AddressFromURIString("akka://sys@darkstar2:1110") +
AddressFromURIString("akka://sys@darkstar1:1110") +
AddressFromURIString("akka://sys@darkstar3:1110") +
AddressFromURIString("akka://sys@darkstar0:1110")
val seq = addresses.toSeq
seq.size must equal(4)
seq(0) must equal(AddressFromURIString("akka://sys@darkstar0:1110"))
seq(1) must equal(AddressFromURIString("akka://sys@darkstar1:1110"))
seq(2) must equal(AddressFromURIString("akka://sys@darkstar2:1110"))
seq(3) must equal(AddressFromURIString("akka://sys@darkstar3:1110"))
}
"order addresses by hostname and port" in {
val addresses = SortedSet.empty[Address] +
AddressFromURIString("akka://sys@darkstar2:1110") +
AddressFromURIString("akka://sys@darkstar0:1111") +
AddressFromURIString("akka://sys@darkstar2:1111") +
AddressFromURIString("akka://sys@darkstar0:1110")
val seq = addresses.toSeq
seq.size must equal(4)
seq(0) must equal(AddressFromURIString("akka://sys@darkstar0:1110"))
seq(1) must equal(AddressFromURIString("akka://sys@darkstar0:1111"))
seq(2) must equal(AddressFromURIString("akka://sys@darkstar2:1110"))
seq(3) must equal(AddressFromURIString("akka://sys@darkstar2:1111"))
}
}
}

View file

@ -1,31 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.actor.Address
import scala.util.Random
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MemberSpec extends WordSpec with MustMatchers {
"Member" must {
"be sorted by address correctly" in {
import Member.ordering
// sorting should be done on host and port, only
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
val m2 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
val m3 = Member(Address("cluster", "sys2", "host2", 8000), MemberStatus.Up)
val m4 = Member(Address("cluster", "sys2", "host2", 9000), MemberStatus.Up)
val m5 = Member(Address("cluster", "sys1", "host2", 10000), MemberStatus.Up)
val expected = IndexedSeq(m1, m2, m3, m4, m5)
val shuffled = Random.shuffle(expected)
shuffled.sorted must be(expected)
}
}
}

View file

@ -27,67 +27,67 @@ class VectorClockSpec extends AkkaSpec {
"pass misc comparison test 1" in {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + Node("1")
val clock3_1 = clock2_1 + Node("2")
val clock4_1 = clock3_1 + Node("1")
val clock2_1 = clock1_1 :+ Node("1")
val clock3_1 = clock2_1 :+ Node("2")
val clock4_1 = clock3_1 :+ Node("1")
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + Node("1")
val clock3_2 = clock2_2 + Node("2")
val clock4_2 = clock3_2 + Node("1")
val clock2_2 = clock1_2 :+ Node("1")
val clock3_2 = clock2_2 :+ Node("2")
val clock4_2 = clock3_2 :+ Node("1")
clock4_1 <> clock4_2 must be(false)
}
"pass misc comparison test 2" in {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + Node("1")
val clock3_1 = clock2_1 + Node("2")
val clock4_1 = clock3_1 + Node("1")
val clock2_1 = clock1_1 :+ Node("1")
val clock3_1 = clock2_1 :+ Node("2")
val clock4_1 = clock3_1 :+ Node("1")
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + Node("1")
val clock3_2 = clock2_2 + Node("2")
val clock4_2 = clock3_2 + Node("1")
val clock5_2 = clock4_2 + Node("3")
val clock2_2 = clock1_2 :+ Node("1")
val clock3_2 = clock2_2 :+ Node("2")
val clock4_2 = clock3_2 :+ Node("1")
val clock5_2 = clock4_2 :+ Node("3")
clock4_1 < clock5_2 must be(true)
}
"pass misc comparison test 3" in {
var clock1_1 = VectorClock()
val clock2_1 = clock1_1 + Node("1")
val clock2_1 = clock1_1 :+ Node("1")
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + Node("2")
val clock2_2 = clock1_2 :+ Node("2")
clock2_1 <> clock2_2 must be(true)
}
"pass misc comparison test 4" in {
val clock1_3 = VectorClock()
val clock2_3 = clock1_3 + Node("1")
val clock3_3 = clock2_3 + Node("2")
val clock4_3 = clock3_3 + Node("1")
val clock2_3 = clock1_3 :+ Node("1")
val clock3_3 = clock2_3 :+ Node("2")
val clock4_3 = clock3_3 :+ Node("1")
val clock1_4 = VectorClock()
val clock2_4 = clock1_4 + Node("1")
val clock3_4 = clock2_4 + Node("1")
val clock4_4 = clock3_4 + Node("3")
val clock2_4 = clock1_4 :+ Node("1")
val clock3_4 = clock2_4 :+ Node("1")
val clock4_4 = clock3_4 :+ Node("3")
clock4_3 <> clock4_4 must be(true)
}
"pass misc comparison test 5" in {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + Node("2")
val clock3_1 = clock2_1 + Node("2")
val clock2_1 = clock1_1 :+ Node("2")
val clock3_1 = clock2_1 :+ Node("2")
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + Node("1")
val clock3_2 = clock2_2 + Node("2")
val clock4_2 = clock3_2 + Node("2")
val clock5_2 = clock4_2 + Node("3")
val clock2_2 = clock1_2 :+ Node("1")
val clock3_2 = clock2_2 :+ Node("2")
val clock4_2 = clock3_2 :+ Node("2")
val clock5_2 = clock4_2 :+ Node("3")
clock3_1 < clock5_2 must be(true)
clock5_2 > clock3_1 must be(true)
@ -95,12 +95,12 @@ class VectorClockSpec extends AkkaSpec {
"pass misc comparison test 6" in {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + Node("1")
val clock3_1 = clock2_1 + Node("2")
val clock2_1 = clock1_1 :+ Node("1")
val clock3_1 = clock2_1 :+ Node("2")
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + Node("1")
val clock3_2 = clock2_2 + Node("1")
val clock2_2 = clock1_2 :+ Node("1")
val clock3_2 = clock2_2 :+ Node("1")
clock3_1 <> clock3_2 must be(true)
clock3_2 <> clock3_1 must be(true)
@ -108,14 +108,14 @@ class VectorClockSpec extends AkkaSpec {
"pass misc comparison test 7" in {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + Node("1")
val clock3_1 = clock2_1 + Node("2")
val clock4_1 = clock3_1 + Node("2")
val clock5_1 = clock4_1 + Node("3")
val clock2_1 = clock1_1 :+ Node("1")
val clock3_1 = clock2_1 :+ Node("2")
val clock4_1 = clock3_1 :+ Node("2")
val clock5_1 = clock4_1 :+ Node("3")
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + Node("2")
val clock3_2 = clock2_2 + Node("2")
val clock2_2 = clock1_2 :+ Node("2")
val clock3_2 = clock2_2 :+ Node("2")
clock5_1 <> clock3_2 must be(true)
clock3_2 <> clock5_1 must be(true)
@ -127,14 +127,14 @@ class VectorClockSpec extends AkkaSpec {
val node3 = Node("3")
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + node1
val clock3_1 = clock2_1 + node2
val clock4_1 = clock3_1 + node2
val clock5_1 = clock4_1 + node3
val clock2_1 = clock1_1 :+ node1
val clock3_1 = clock2_1 :+ node2
val clock4_1 = clock3_1 :+ node2
val clock5_1 = clock4_1 :+ node3
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + node2
val clock3_2 = clock2_2 + node2
val clock2_2 = clock1_2 :+ node2
val clock3_2 = clock2_2 :+ node2
val merged1 = clock3_2 merge clock5_1
merged1.versions.size must be(3)
@ -164,14 +164,14 @@ class VectorClockSpec extends AkkaSpec {
val node4 = Node("4")
val clock1_1 = VectorClock()
val clock2_1 = clock1_1 + node1
val clock3_1 = clock2_1 + node2
val clock4_1 = clock3_1 + node2
val clock5_1 = clock4_1 + node3
val clock2_1 = clock1_1 :+ node1
val clock3_1 = clock2_1 :+ node2
val clock4_1 = clock3_1 :+ node2
val clock5_1 = clock4_1 :+ node3
val clock1_2 = VectorClock()
val clock2_2 = clock1_2 + node4
val clock3_2 = clock2_2 + node4
val clock2_2 = clock1_2 :+ node4
val clock3_2 = clock2_2 :+ node4
val merged1 = clock3_2 merge clock5_1
merged1.versions.size must be(4)
@ -204,8 +204,8 @@ class VectorClockSpec extends AkkaSpec {
val v1 = VectorClock()
val v2 = VectorClock()
val vv1 = v1 + node1
val vv2 = v2 + node2
val vv1 = v1 :+ node1
val vv2 = v2 :+ node2
(vv1 > v1) must equal(true)
(vv2 > v2) must equal(true)
@ -225,12 +225,12 @@ class VectorClockSpec extends AkkaSpec {
val a = VectorClock()
val b = VectorClock()
val a1 = a + node1
val b1 = b + node2
val a1 = a :+ node1
val b1 = b :+ node2
var a2 = a1 + node1
var a2 = a1 :+ node1
var c = a2.merge(b1)
var c1 = c + node3
var c1 = c :+ node3
(c1 > a2) must equal(true)
(c1 > b1) must equal(true)
@ -239,7 +239,7 @@ class VectorClockSpec extends AkkaSpec {
"An instance of Versioned" must {
class TestVersioned(val version: VectorClock = VectorClock()) extends Versioned[TestVersioned] {
def +(node: Node): TestVersioned = new TestVersioned(version + node)
def :+(node: Node): TestVersioned = new TestVersioned(version :+ node)
}
import Versioned.latestVersionOf
@ -251,67 +251,67 @@ class VectorClockSpec extends AkkaSpec {
"happen before an identical versioned with a single additional event" in {
val versioned1_1 = new TestVersioned()
val versioned2_1 = versioned1_1 + Node("1")
val versioned3_1 = versioned2_1 + Node("2")
val versioned4_1 = versioned3_1 + Node("1")
val versioned2_1 = versioned1_1 :+ Node("1")
val versioned3_1 = versioned2_1 :+ Node("2")
val versioned4_1 = versioned3_1 :+ Node("1")
val versioned1_2 = new TestVersioned()
val versioned2_2 = versioned1_2 + Node("1")
val versioned3_2 = versioned2_2 + Node("2")
val versioned4_2 = versioned3_2 + Node("1")
val versioned5_2 = versioned4_2 + Node("3")
val versioned2_2 = versioned1_2 :+ Node("1")
val versioned3_2 = versioned2_2 :+ Node("2")
val versioned4_2 = versioned3_2 :+ Node("1")
val versioned5_2 = versioned4_2 :+ Node("3")
latestVersionOf[TestVersioned](versioned4_1, versioned5_2) must be(versioned5_2)
}
"pass misc comparison test 1" in {
var versioned1_1 = new TestVersioned()
val versioned2_1 = versioned1_1 + Node("1")
val versioned2_1 = versioned1_1 :+ Node("1")
val versioned1_2 = new TestVersioned()
val versioned2_2 = versioned1_2 + Node("2")
val versioned2_2 = versioned1_2 :+ Node("2")
latestVersionOf[TestVersioned](versioned2_1, versioned2_2) must be(versioned2_2)
}
"pass misc comparison test 2" in {
val versioned1_3 = new TestVersioned()
val versioned2_3 = versioned1_3 + Node("1")
val versioned3_3 = versioned2_3 + Node("2")
val versioned4_3 = versioned3_3 + Node("1")
val versioned2_3 = versioned1_3 :+ Node("1")
val versioned3_3 = versioned2_3 :+ Node("2")
val versioned4_3 = versioned3_3 :+ Node("1")
val versioned1_4 = new TestVersioned()
val versioned2_4 = versioned1_4 + Node("1")
val versioned3_4 = versioned2_4 + Node("1")
val versioned4_4 = versioned3_4 + Node("3")
val versioned2_4 = versioned1_4 :+ Node("1")
val versioned3_4 = versioned2_4 :+ Node("1")
val versioned4_4 = versioned3_4 :+ Node("3")
latestVersionOf[TestVersioned](versioned4_3, versioned4_4) must be(versioned4_4)
}
"pass misc comparison test 3" in {
val versioned1_1 = new TestVersioned()
val versioned2_1 = versioned1_1 + Node("2")
val versioned3_1 = versioned2_1 + Node("2")
val versioned2_1 = versioned1_1 :+ Node("2")
val versioned3_1 = versioned2_1 :+ Node("2")
val versioned1_2 = new TestVersioned()
val versioned2_2 = versioned1_2 + Node("1")
val versioned3_2 = versioned2_2 + Node("2")
val versioned4_2 = versioned3_2 + Node("2")
val versioned5_2 = versioned4_2 + Node("3")
val versioned2_2 = versioned1_2 :+ Node("1")
val versioned3_2 = versioned2_2 :+ Node("2")
val versioned4_2 = versioned3_2 :+ Node("2")
val versioned5_2 = versioned4_2 :+ Node("3")
latestVersionOf[TestVersioned](versioned3_1, versioned5_2) must be(versioned5_2)
}
"pass misc comparison test 4" in {
val versioned1_1 = new TestVersioned()
val versioned2_1 = versioned1_1 + Node("1")
val versioned3_1 = versioned2_1 + Node("2")
val versioned4_1 = versioned3_1 + Node("2")
val versioned5_1 = versioned4_1 + Node("3")
val versioned2_1 = versioned1_1 :+ Node("1")
val versioned3_1 = versioned2_1 :+ Node("2")
val versioned4_1 = versioned3_1 :+ Node("2")
val versioned5_1 = versioned4_1 :+ Node("3")
val versioned1_2 = new TestVersioned()
val versioned2_2 = versioned1_2 + Node("2")
val versioned3_2 = versioned2_2 + Node("2")
val versioned2_2 = versioned1_2 :+ Node("2")
val versioned3_2 = versioned2_2 :+ Node("2")
latestVersionOf[TestVersioned](versioned5_1, versioned3_2) must be(versioned3_2)
}

View file

@ -90,6 +90,42 @@ strong {color: #1d3c52; }
box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
}
.warning {
background-image: none;
background-color: #fdf5d9;
filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);
padding: 14px;
border-color: #ffffc4;
-webkit-box-shadow: none;
-moz-box-shadow: none;
box-shadow: none;
margin-bottom: 18px;
position: relative;
padding: 7px 15px;
color: #404040;
background-repeat: repeat-x;
background-image: -khtml-gradient(linear, left top, left bottom, from(#ffffc4), to(#ffff00));
background-image: -moz-linear-gradient(top, #ffffc4, #ffff00);
background-image: -ms-linear-gradient(top, #ffffc4, #ffff00);
background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #ffffc4), color-stop(100%, #ffff00));
background-image: -webkit-linear-gradient(top, #ffffc4, #ffff00);
background-image: -o-linear-gradient(top, #ffffc4, #ffff00);
background-image: linear-gradient(top, #ffffc4, #ffff00);
filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffc4', endColorstr='#ffff00', GradientType=0);
text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
border-color: #dff69a #ffff00 #E4C652;
border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);
text-shadow: 0 1px 0 rgba(255, 255, 255, 0.5);
border-width: 1px;
border-style: solid;
-webkit-border-radius: 4px;
-moz-border-radius: 4px;
border-radius: 4px;
-webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
-moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
}
.admonition p.admonition-title {
color: rgba(0, 0, 0, 0.6);
text-shadow: 0 1px 0 rgba(255, 255, 255, .7);

View file

@ -5,7 +5,7 @@
Cluster Specification
######################
.. note:: *This document describes the new clustering coming in Akka 2.1 (not 2.0)*
.. note:: *This document describes the new clustering coming in Akka Coltrane and is not available in the latest stable release)*
Intro
=====
@ -81,16 +81,6 @@ can later explicitly send a ``Join`` message to another node to form a N-node
cluster. It is also possible to link multiple N-node clusters by ``joining`` them.
Singleton Cluster
-----------------
If a node does not have a preconfigured contact point to join in the Akka
configuration, then it is considered a singleton cluster (single node cluster)
and will automatically transition from ``joining`` to ``up``. Singleton clusters
can later explicitly send a ``Join`` message to another node to form a N-node
cluster. It is also possible to link multiple N-node clusters by ``joining`` them.
Gossip
------
@ -173,8 +163,8 @@ After gossip convergence a ``leader`` for the cluster can be determined. There i
``leader`` election process, the ``leader`` can always be recognised deterministically
by any node whenever there is gossip convergence. The ``leader`` is simply the first
node in sorted order that is able to take the leadership role, where the only
allowed member states for a ``leader`` are ``up`` or ``leaving`` (see below for more
information about member states).
allowed member states for a ``leader`` are ``up``, ``leaving`` or ``exiting`` (see
below for more information about member states).
The role of the ``leader`` is to shift members in and out of the cluster, changing
``joining`` members to the ``up`` state or ``exiting`` members to the
@ -311,10 +301,6 @@ handoff has completed then the node will change to the ``exiting`` state. Once
all nodes have seen the exiting state (convergence) the ``leader`` will remove the
node from the cluster, marking it as ``removed``.
A node can also be removed forcefully by moving it directly to the ``removed``
state using the ``remove`` action. The cluster will rebalance based on the new
cluster membership.
If a node is unreachable then gossip convergence is not possible and therefore
any ``leader`` actions are also not possible (for instance, allowing a node to
become a part of the cluster, or changing actor distribution). To be able to
@ -323,11 +309,12 @@ unreachable node is experiencing only transient difficulties then it can be
explicitly marked as ``down`` using the ``down`` user action. When this node
comes back up and begins gossiping it will automatically go through the joining
process again. If the unreachable node will be permanently down then it can be
removed from the cluster directly with the ``remove`` user action. The cluster
can also *auto-down* a node using the accrual failure detector.
removed from the cluster directly by shutting the actor system down or killing it
through an external ``SIGKILL`` signal, invocation of ``System.exit(status)`` or
similar. The cluster can, through the leader, also *auto-down* a node.
This means that nodes can join and leave the cluster at any point in time,
e.g. provide cluster elasticity.
This means that nodes can join and leave the cluster at any point in time, i.e.
provide cluster elasticity.
State Diagram for the Member States
@ -348,12 +335,12 @@ Member States
- **leaving** / **exiting**
states during graceful removal
- **removed**
tombstone state (no longer a member)
- **down**
marked as down/offline/unreachable
- **removed**
tombstone state (no longer a member)
User Actions
^^^^^^^^^^^^
@ -368,9 +355,6 @@ User Actions
- **down**
mark a node as temporarily down
- **remove**
remove a node from the cluster immediately
Leader Actions
^^^^^^^^^^^^^^

View file

@ -0,0 +1,130 @@
.. _circuit-breaker:
###############
Circuit Breaker
###############
==================
Why are they used?
==================
A circuit breaker is used to provide stability and prevent cascading failures in distributed
systems. These should be used in conjunction with judicious timeouts at the interfaces between
remote systems to prevent the failure of a single component from bringing down all components.
As an example, we have a web application interacting with a remote third party web service.
Let's say the third party has oversold their capacity and their database melts down under load.
Assume that the database fails in such a way that it takes a very long time to hand back an
error to the third party web service. This in turn makes calls fail after a long period of
time. Back to our web application, the users have noticed that their form submissions take
much longer seeming to hang. Well the users do what they know to do which is use the refresh
button, adding more requests to their already running requests. This eventually causes the
failure of the web application due to resource exhaustion. This will affect all users, even
those who are not using functionality dependent on this third party web service.
Introducing circuit breakers on the web service call would cause the requests to begin to
fail-fast, letting the user know that something is wrong and that they need not refresh
their request. This also confines the failure behavior to only those users that are using
functionality dependent on the third party, other users are no longer affected as there is no
resource exhaustion. Circuit breakers can also allow savvy developers to mark portions of
the site that use the functionality unavailable, or perhaps show some cached content as
appropriate while the breaker is open.
The Akka library provides an implementation of a circuit breaker called
:class:`akka.pattern.CircuitBreaker` which has the behavior described below.
=================
What do they do?
=================
* During normal operation, a circuit breaker is in the `Closed` state:
* Exceptions or calls exceeding the configured `callTimeout` increment a failure counter
* Successes reset the failure count to zero
* When the failure counter reaches a `maxFailures` count, the breaker is tripped into `Open` state
* While in `Open` state:
* All calls fail-fast with a :class:`CircuitBreakerOpenException`
* After the configured `resetTimeout`, the circuit breaker enters a `Half-Open` state
* In `Half-Open` state:
* The first call attempted is allowed through without failing fast
* All other calls fail-fast with an exception just as in `Open` state
* If the first call succeeds, the breaker is reset back to `Closed` state
* If the first call fails, the breaker is tripped again into the `Open` state for another full `resetTimeout`
* State transition listeners:
* Callbacks can be provided for every state entry via `onOpen`, `onClose`, and `onHalfOpen`
* These are executed in the :class:`ExecutionContext` provided.
.. graphviz::
digraph circuit_breaker {
rankdir = "LR";
size = "6,5";
graph [ bgcolor = "transparent" ]
node [ fontname = "Helvetica",
fontsize = 14,
shape = circle,
color = white,
style = filled ];
edge [ fontname = "Helvetica", fontsize = 12 ]
Closed [ fillcolor = green2 ];
"Half-Open" [fillcolor = yellow2 ];
Open [ fillcolor = red2 ];
Closed -> Closed [ label = "Success" ];
"Half-Open" -> Open [ label = "Trip Breaker" ];
"Half-Open" -> Closed [ label = "Reset Breaker" ];
Closed -> Open [ label = "Trip Breaker" ];
Open -> Open [ label = "Calls failing fast" ];
Open -> "Half-Open" [ label = "Attempt Reset" ];
}
========
Examples
========
--------------
Initialization
--------------
Here's how a :class:`CircuitBreaker` would be configured for:
* 5 maximum failures
* a call timeout of 10 seconds
* a reset timeout of 1 minute
^^^^^^^
Scala
^^^^^^^
.. includecode:: code/docs/circuitbreaker/CircuitBreakerDocSpec.scala
:include: imports1,circuit-breaker-initialization
^^^^^^^
Java
^^^^^^^
.. includecode:: code/docs/circuitbreaker/DangerousJavaActor.java
:include: imports1,circuit-breaker-initialization
---------------
Call Protection
---------------
Here's how the :class:`CircuitBreaker` would be used to protect an asynchronous
call as well as a synchronous one:
^^^^^^^
Scala
^^^^^^^
.. includecode:: code/docs/circuitbreaker/CircuitBreakerDocSpec.scala
:include: circuit-breaker-usage
^^^^^^
Java
^^^^^^
.. includecode:: code/docs/circuitbreaker/DangerousJavaActor.java
:include: circuit-breaker-usage
.. note::
Using the :class:`CircuitBreaker` companion object's `apply` or `create` methods
will return a :class:`CircuitBreaker` where callbacks are executed in the caller's thread.
This can be useful if the asynchronous :class:`Future` behavior is unnecessary, for
example invoking a synchronous-only API.

View file

@ -0,0 +1,43 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.circuitbreaker
//#imports1
import akka.util.duration._ // small d is important here
import akka.pattern.CircuitBreaker
import akka.actor.Actor
import akka.dispatch.Future
import akka.event.Logging
//#imports1
class CircuitBreakerDocSpec {}
//#circuit-breaker-initialization
class DangerousActor extends Actor {
val log = Logging(context.system, this)
implicit val executionContext = context.dispatcher
val breaker =
new CircuitBreaker(context.system.scheduler, 5, 10.seconds, 1.minute)
.onOpen(notifyMeOnOpen)
def notifyMeOnOpen =
log.warning("My CircuitBreaker is now open, and will not close for one minute")
//#circuit-breaker-initialization
//#circuit-breaker-usage
def dangerousCall: String = "This really isn't that dangerous of a call after all"
def receive = {
case "is my middle name"
sender ! breaker.withCircuitBreaker(Future(dangerousCall))
case "block for me"
sender ! breaker.withSyncCircuitBreaker(dangerousCall)
}
//#circuit-breaker-usage
}

View file

@ -0,0 +1,83 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.circuitbreaker;
//#imports1
import akka.actor.UntypedActor;
import akka.dispatch.Future;
import akka.event.LoggingAdapter;
import akka.util.Duration;
import akka.pattern.CircuitBreaker;
import akka.event.Logging;
import static akka.dispatch.Futures.future;
import java.util.concurrent.Callable;
//#imports1
//#circuit-breaker-initialization
public class DangerousJavaActor extends UntypedActor {
private final CircuitBreaker breaker;
private final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
public DangerousJavaActor() {
this.breaker = new CircuitBreaker(
getContext().dispatcher(), getContext().system().scheduler(),
5, Duration.parse("10s"), Duration.parse("1m"))
.onOpen(new Callable<Object>() {
public Object call() throws Exception {
notifyMeOnOpen();
return null;
}
});
}
public void notifyMeOnOpen() {
log.warning("My CircuitBreaker is now open, and will not close for one minute");
}
//#circuit-breaker-initialization
//#circuit-breaker-usage
public String dangerousCall() {
return "This really isn't that dangerous of a call after all";
}
@Override
public void onReceive(Object message) {
if (message instanceof String) {
String m = (String) message;
if ("is my middle name".equals(m)) {
final Future<String> f = future(
new Callable<String>() {
public String call() {
return dangerousCall();
}
}, getContext().dispatcher());
getSender().tell(breaker
.callWithCircuitBreaker(
new Callable<Future<String>>() {
public Future<String> call() throws Exception {
return f;
}
}));
}
if ("block for me".equals(m)) {
getSender().tell(breaker
.callWithSyncCircuitBreaker(
new Callable<String>() {
@Override
public String call() throws Exception {
return dangerousCall();
}
}));
}
}
}
//#circuit-breaker-usage
}

View file

@ -5,3 +5,4 @@ Common utilities
:maxdepth: 2
duration
circuitbreaker

View file

@ -8,7 +8,7 @@ import sys, os
# -- General configuration -----------------------------------------------------
sys.path.append(os.path.abspath('_sphinx/exts'))
extensions = ['sphinx.ext.todo', 'includecode']
extensions = ['sphinx.ext.todo', 'includecode', 'sphinx.ext.graphviz']
templates_path = ['_templates']
source_suffix = '.rst'

Binary file not shown.

Before

Width:  |  Height:  |  Size: 50 KiB

After

Width:  |  Height:  |  Size: 41 KiB

Before After
Before After

View file

@ -65,7 +65,7 @@ When using JarJar, OneJar, Assembly or any jar-bundler
Akka's configuration approach relies heavily on the notion of every
module/jar having its own reference.conf file, all of these will be
discovered by the configuration and loaded. Unfortunately this also means
that if you put merge multiple jars into the same jar, you need to merge all the
that if you put/merge multiple jars into the same jar, you need to merge all the
reference.confs as well. Otherwise all defaults will be lost and Akka will not function.
Custom application.conf

View file

@ -55,7 +55,7 @@ which is a user-level concern.
Ordering is preserved on a per-sender basis
-------------------------------------------
Actor ``A1` sends messages ``M1``, ``M2``, ``M3`` to ``A2``
Actor ``A1`` sends messages ``M1``, ``M2``, ``M3`` to ``A2``
Actor ``A3`` sends messages ``M4``, ``M5``, ``M6`` to ``A2``
This means that:

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