Merge branch 'master' into wip-2212-recursive-resume-∂π

also split out ChildrenContainer into its own file and add
suspendCounter to UnstartedActorCell
This commit is contained in:
Roland 2012-07-05 13:18:19 +02:00
commit 1cb204da49
194 changed files with 9705 additions and 2308 deletions

1
.gitignore vendored
View file

@ -64,3 +64,4 @@ mongoDB/
redis/
beanstalk/
.scalastyle
bin/

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

@ -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 5 sender.tell("five")
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

@ -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,13 +140,13 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
object FSMTimingSpec {
def suspend(actorRef: ActorRef): Unit = actorRef match {
case l: LocalActorRef l.suspend()
case _
case l: ActorRefWithCell l.suspend()
case _
}
def resume(actorRef: ActorRef): Unit = actorRef match {
case l: LocalActorRef l.resume(inResponseToFailure = false)
case _
case l: ActorRefWithCell l.resume(inResponseToFailure = false)
case _
}
trait State

View file

@ -55,19 +55,19 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
actor4 ! Kill
countDownLatch.await(10, TimeUnit.SECONDS)
assert(Await.result(actor1 ? "status", timeout.duration) == "OK", "actor1 is shutdown")
assert(Await.result(actor2 ? "status", timeout.duration) == "OK", "actor2 is shutdown")
assert(Await.result(actor3 ? "status", timeout.duration) == "OK", "actor3 is shutdown")
assert(Await.result(actor4 ? "status", timeout.duration) == "OK", "actor4 is shutdown")
Seq("actor1" -> actor1, "actor2" -> actor2, "actor3" -> actor3, "actor4" -> actor4) map {
case (id, ref) (id, ref ? "status")
} foreach {
case (id, f) (id, Await.result(f, timeout.duration)) must be === ((id, "OK"))
}
}
}
"be able to create named children in its constructor" in {
val a = system.actorOf(Props(new Actor {
context.actorOf(Props.empty, "bob")
def receive = {
case x: Exception throw x
}
def receive = { case x: Exception throw x }
override def preStart(): Unit = testActor ! "preStart"
}))
val m = "weird message"
@ -123,20 +123,14 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
"be able to create a similar kid in the fault handling strategy" in {
val parent = system.actorOf(Props(new Actor {
override val supervisorStrategy = new OneForOneStrategy()(SupervisorStrategy.defaultStrategy.decider) {
override def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = {
val newKid = context.actorOf(Props.empty, child.path.name)
testActor ! {
if ((newKid ne child) && newKid.path == child.path) "green"
else "red"
}
testActor ! { if ((newKid ne child) && newKid.path == child.path) "green" else "red" }
}
}
def receive = {
case "engage" context.stop(context.actorOf(Props.empty, "Robert"))
}
def receive = { case "engage" context.stop(context.actorOf(Props.empty, "Robert")) }
}))
parent ! "engage"
expectMsg("green")

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

@ -802,33 +802,6 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
assert(Await.result(z, timeout.duration) === 42)
}
"futureFlowLoops" in {
import Future.flow
import akka.util.cps._
val count = 1000
val promises = List.fill(count)(Promise[Int]())
flow {
var i = 0
val iter = promises.iterator
whileC(iter.hasNext) {
iter.next << i
i += 1
}
}
var i = 0
promises foreach { p
assert(Await.result(p, timeout.duration) === i)
i += 1
}
assert(i === count)
}
"run callbacks async" in {
val latch = Vector.fill(10)(new TestLatch)

View file

@ -1,13 +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 java.util.concurrent.{ ConcurrentLinkedQueue, BlockingQueue }
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll }
import com.typesafe.config.Config
import akka.actor._
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 {
@ -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

@ -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 {
@ -25,6 +26,10 @@ object RoutingSpec {
router = round-robin
nr-of-instances = 3
}
/router2 {
router = round-robin
nr-of-instances = 3
}
/myrouter {
router = "akka.routing.RoutingSpec$MyRouter"
foo = bar
@ -128,7 +133,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
}
"use configured nr-of-instances when router is specified" in {
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router1")
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2")
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(3)
system.stop(router)
}
@ -171,6 +176,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 {
@ -528,7 +545,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)
}
}
@ -540,7 +557,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

@ -9,12 +9,21 @@ import org.scalacheck.Arbitrary._
import org.scalacheck.Prop._
import org.scalacheck.Gen._
import scala.collection.mutable.Builder
import java.nio.{ ByteBuffer, ShortBuffer, IntBuffer, FloatBuffer, DoubleBuffer }
import java.nio.ByteOrder, ByteOrder.{ BIG_ENDIAN, LITTLE_ENDIAN }
import java.lang.Float.floatToRawIntBits
import java.lang.Double.doubleToRawLongBits
class ByteStringSpec extends WordSpec with MustMatchers with Checkers {
def genSimpleByteString(min: Int, max: Int) = for {
n choose(min, max)
b Gen.containerOfN[Array, Byte](n, arbitrary[Byte])
} yield ByteString(b)
from choose(0, b.length)
until choose(from, b.length)
} yield ByteString(b).slice(from, until)
implicit val arbitraryByteString: Arbitrary[ByteString] = Arbitrary {
Gen.sized { s
@ -25,14 +34,483 @@ class ByteStringSpec extends WordSpec with MustMatchers with Checkers {
}
}
type ByteStringSlice = (ByteString, Int, Int)
implicit val arbitraryByteStringSlice: Arbitrary[ByteStringSlice] = Arbitrary {
for {
xs arbitraryByteString.arbitrary
from choose(0, xs.length)
until choose(from, xs.length)
} yield (xs, from, until)
}
type ArraySlice[A] = (Array[A], Int, Int)
def arbSlice[A](arbArray: Arbitrary[Array[A]]): Arbitrary[ArraySlice[A]] = Arbitrary {
for {
xs arbArray.arbitrary
from choose(0, xs.length)
until choose(from, xs.length)
} yield (xs, from, until)
}
val arbitraryByteArray: Arbitrary[Array[Byte]] = Arbitrary { Gen.sized { n Gen.containerOfN[Array, Byte](n, arbitrary[Byte]) } }
implicit val arbitraryByteArraySlice: Arbitrary[ArraySlice[Byte]] = arbSlice(arbitraryByteArray)
val arbitraryShortArray: Arbitrary[Array[Short]] = Arbitrary { Gen.sized { n Gen.containerOfN[Array, Short](n, arbitrary[Short]) } }
implicit val arbitraryShortArraySlice: Arbitrary[ArraySlice[Short]] = arbSlice(arbitraryShortArray)
val arbitraryIntArray: Arbitrary[Array[Int]] = Arbitrary { Gen.sized { n Gen.containerOfN[Array, Int](n, arbitrary[Int]) } }
implicit val arbitraryIntArraySlice: Arbitrary[ArraySlice[Int]] = arbSlice(arbitraryIntArray)
val arbitraryLongArray: Arbitrary[Array[Long]] = Arbitrary { Gen.sized { n Gen.containerOfN[Array, Long](n, arbitrary[Long]) } }
implicit val arbitraryLongArraySlice: Arbitrary[ArraySlice[Long]] = arbSlice(arbitraryLongArray)
val arbitraryFloatArray: Arbitrary[Array[Float]] = Arbitrary { Gen.sized { n Gen.containerOfN[Array, Float](n, arbitrary[Float]) } }
implicit val arbitraryFloatArraySlice: Arbitrary[ArraySlice[Float]] = arbSlice(arbitraryFloatArray)
val arbitraryDoubleArray: Arbitrary[Array[Double]] = Arbitrary { Gen.sized { n Gen.containerOfN[Array, Double](n, arbitrary[Double]) } }
implicit val arbitraryDoubleArraySlice: Arbitrary[ArraySlice[Double]] = arbSlice(arbitraryDoubleArray)
def likeVector(bs: ByteString)(body: IndexedSeq[Byte] Any): Boolean = {
val vec = Vector(bs: _*)
body(bs) == body(vec)
}
def likeVectors(bsA: ByteString, bsB: ByteString)(body: (IndexedSeq[Byte], IndexedSeq[Byte]) Any): Boolean = {
val vecA = Vector(bsA: _*)
val vecB = Vector(bsB: _*)
body(bsA, bsB) == body(vecA, vecB)
}
def likeVecIt(bs: ByteString)(body: BufferedIterator[Byte] Any, strict: Boolean = true): Boolean = {
val bsIterator = bs.iterator
val vecIterator = Vector(bs: _*).iterator.buffered
(body(bsIterator) == body(vecIterator)) &&
(!strict || (bsIterator.toSeq == vecIterator.toSeq))
}
def likeVecIts(a: ByteString, b: ByteString)(body: (BufferedIterator[Byte], BufferedIterator[Byte]) Any, strict: Boolean = true): Boolean = {
val (bsAIt, bsBIt) = (a.iterator, b.iterator)
val (vecAIt, vecBIt) = (Vector(a: _*).iterator.buffered, Vector(b: _*).iterator.buffered)
(body(bsAIt, bsBIt) == body(vecAIt, vecBIt)) &&
(!strict || (bsAIt.toSeq, bsBIt.toSeq) == (vecAIt.toSeq, vecBIt.toSeq))
}
def likeVecBld(body: Builder[Byte, _] Unit): Boolean = {
val bsBuilder = ByteString.newBuilder
val vecBuilder = Vector.newBuilder[Byte]
body(bsBuilder)
body(vecBuilder)
bsBuilder.result == vecBuilder.result
}
def testShortDecoding(slice: ByteStringSlice, byteOrder: ByteOrder): Boolean = {
val elemSize = 2
val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Short](n)
bytes.asByteBuffer.order(byteOrder).asShortBuffer.get(reference, 0, n)
val input = bytes.iterator
val decoded = Array.ofDim[Short](n)
for (i 0 to a - 1) decoded(i) = input.getShort(byteOrder)
input.getShorts(decoded, a, b - a)(byteOrder)
for (i b to n - 1) decoded(i) = input.getShort(byteOrder)
(decoded.toSeq == reference.toSeq) && (input.toSeq == bytes.drop(n * elemSize))
}
def testIntDecoding(slice: ByteStringSlice, byteOrder: ByteOrder): Boolean = {
val elemSize = 4
val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Int](n)
bytes.asByteBuffer.order(byteOrder).asIntBuffer.get(reference, 0, n)
val input = bytes.iterator
val decoded = Array.ofDim[Int](n)
for (i 0 to a - 1) decoded(i) = input.getInt(byteOrder)
input.getInts(decoded, a, b - a)(byteOrder)
for (i b to n - 1) decoded(i) = input.getInt(byteOrder)
(decoded.toSeq == reference.toSeq) && (input.toSeq == bytes.drop(n * elemSize))
}
def testLongDecoding(slice: ByteStringSlice, byteOrder: ByteOrder): Boolean = {
val elemSize = 8
val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Long](n)
bytes.asByteBuffer.order(byteOrder).asLongBuffer.get(reference, 0, n)
val input = bytes.iterator
val decoded = Array.ofDim[Long](n)
for (i 0 to a - 1) decoded(i) = input.getLong(byteOrder)
input.getLongs(decoded, a, b - a)(byteOrder)
for (i b to n - 1) decoded(i) = input.getLong(byteOrder)
(decoded.toSeq == reference.toSeq) && (input.toSeq == bytes.drop(n * elemSize))
}
def testFloatDecoding(slice: ByteStringSlice, byteOrder: ByteOrder): Boolean = {
val elemSize = 4
val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Float](n)
bytes.asByteBuffer.order(byteOrder).asFloatBuffer.get(reference, 0, n)
val input = bytes.iterator
val decoded = Array.ofDim[Float](n)
for (i 0 to a - 1) decoded(i) = input.getFloat(byteOrder)
input.getFloats(decoded, a, b - a)(byteOrder)
for (i b to n - 1) decoded(i) = input.getFloat(byteOrder)
((decoded.toSeq map floatToRawIntBits) == (reference.toSeq map floatToRawIntBits)) &&
(input.toSeq == bytes.drop(n * elemSize))
}
def testDoubleDecoding(slice: ByteStringSlice, byteOrder: ByteOrder): Boolean = {
val elemSize = 8
val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Double](n)
bytes.asByteBuffer.order(byteOrder).asDoubleBuffer.get(reference, 0, n)
val input = bytes.iterator
val decoded = Array.ofDim[Double](n)
for (i 0 to a - 1) decoded(i) = input.getDouble(byteOrder)
input.getDoubles(decoded, a, b - a)(byteOrder)
for (i b to n - 1) decoded(i) = input.getDouble(byteOrder)
((decoded.toSeq map doubleToRawLongBits) == (reference.toSeq map doubleToRawLongBits)) &&
(input.toSeq == bytes.drop(n * elemSize))
}
def testShortEncoding(slice: ArraySlice[Short], byteOrder: ByteOrder): Boolean = {
val elemSize = 2
val (data, from, until) = slice
val reference = Array.ofDim[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asShortBuffer.put(data)
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.putShort(data(i))(byteOrder)
builder.putShorts(data, from, until - from)(byteOrder)
for (i until to data.length - 1) builder.putShort(data(i))(byteOrder)
reference.toSeq == builder.result
}
def testIntEncoding(slice: ArraySlice[Int], byteOrder: ByteOrder): Boolean = {
val elemSize = 4
val (data, from, until) = slice
val reference = Array.ofDim[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asIntBuffer.put(data)
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.putInt(data(i))(byteOrder)
builder.putInts(data, from, until - from)(byteOrder)
for (i until to data.length - 1) builder.putInt(data(i))(byteOrder)
reference.toSeq == builder.result
}
def testLongEncoding(slice: ArraySlice[Long], byteOrder: ByteOrder): Boolean = {
val elemSize = 8
val (data, from, until) = slice
val reference = Array.ofDim[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asLongBuffer.put(data)
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.putLong(data(i))(byteOrder)
builder.putLongs(data, from, until - from)(byteOrder)
for (i until to data.length - 1) builder.putLong(data(i))(byteOrder)
reference.toSeq == builder.result
}
def testFloatEncoding(slice: ArraySlice[Float], byteOrder: ByteOrder): Boolean = {
val elemSize = 4
val (data, from, until) = slice
val reference = Array.ofDim[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asFloatBuffer.put(data)
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.putFloat(data(i))(byteOrder)
builder.putFloats(data, from, until - from)(byteOrder)
for (i until to data.length - 1) builder.putFloat(data(i))(byteOrder)
reference.toSeq == builder.result
}
def testDoubleEncoding(slice: ArraySlice[Double], byteOrder: ByteOrder): Boolean = {
val elemSize = 8
val (data, from, until) = slice
val reference = Array.ofDim[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asDoubleBuffer.put(data)
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.putDouble(data(i))(byteOrder)
builder.putDoubles(data, from, until - from)(byteOrder)
for (i until to data.length - 1) builder.putDouble(data(i))(byteOrder)
reference.toSeq == builder.result
}
"A ByteString" must {
"have correct size" when {
"concatenating" in { check((a: ByteString, b: ByteString) (a ++ b).size == a.size + b.size) }
"dropping" in { check((a: ByteString, b: ByteString) (a ++ b).drop(b.size).size == a.size) }
}
"be sequential" when {
"taking" in { check((a: ByteString, b: ByteString) (a ++ b).take(a.size) == a) }
"dropping" in { check((a: ByteString, b: ByteString) (a ++ b).drop(a.size) == b) }
}
"be equal to the original" when {
"compacting" in { check { xs: ByteString val ys = xs.compact; (xs == ys) && ys.isCompact } }
"recombining" in {
check { (xs: ByteString, from: Int, until: Int)
val (tmp, c) = xs.splitAt(until)
val (a, b) = tmp.splitAt(from)
(a ++ b ++ c) == xs
}
}
}
"behave as expected" when {
"created from and decoding to String" in { check { s: String ByteString(s, "UTF-8").decodeString("UTF-8") == s } }
"compacting" in {
check { a: ByteString
val wasCompact = a.isCompact
val b = a.compact
((!wasCompact) || (b eq a)) &&
(b == a) &&
b.isCompact &&
(b.compact eq b)
}
}
}
"behave like a Vector" when {
"concatenating" in { check { (a: ByteString, b: ByteString) likeVectors(a, b) { (a, b) (a ++ b) } } }
"calling apply" in {
check { slice: ByteStringSlice
slice match {
case (xs, i1, i2) likeVector(xs) { seq
(if ((i1 >= 0) && (i1 < seq.length)) seq(i1) else 0,
if ((i2 >= 0) && (i2 < seq.length)) seq(i2) else 0)
}
}
}
}
"calling head" in { check { a: ByteString a.isEmpty || likeVector(a) { _.head } } }
"calling tail" in { check { a: ByteString a.isEmpty || likeVector(a) { _.tail } } }
"calling last" in { check { a: ByteString a.isEmpty || likeVector(a) { _.last } } }
"calling init" in { check { a: ByteString a.isEmpty || likeVector(a) { _.init } } }
"calling length" in { check { a: ByteString likeVector(a) { _.length } } }
"calling span" in { check { (a: ByteString, b: Byte) likeVector(a)({ _.span(_ != b) match { case (a, b) (a, b) } }) } }
"calling takeWhile" in { check { (a: ByteString, b: Byte) likeVector(a)({ _.takeWhile(_ != b) }) } }
"calling dropWhile" in { check { (a: ByteString, b: Byte) likeVector(a) { _.dropWhile(_ != b) } } }
"calling indexWhere" in { check { (a: ByteString, b: Byte) likeVector(a) { _.indexWhere(_ == b) } } }
"calling indexOf" in { check { (a: ByteString, b: Byte) likeVector(a) { _.indexOf(b) } } }
"calling foreach" in { check { a: ByteString likeVector(a) { it var acc = 0; it foreach { acc += _ }; acc } } }
"calling foldLeft" in { check { a: ByteString likeVector(a) { _.foldLeft(0) { _ + _ } } } }
"calling toArray" in { check { a: ByteString likeVector(a) { _.toArray.toSeq } } }
"calling slice" in {
check { slice: ByteStringSlice
slice match {
case (xs, from, until) likeVector(xs)({
_.slice(from, until)
})
}
}
}
"calling take and drop" in {
check { slice: ByteStringSlice
slice match {
case (xs, from, until) likeVector(xs)({
_.drop(from).take(until - from)
})
}
}
}
"calling copyToArray" in {
check { slice: ByteStringSlice
slice match {
case (xs, from, until) likeVector(xs)({ it
val array = Array.ofDim[Byte](xs.length)
it.slice(from, until).copyToArray(array, from, until)
array.toSeq
})
}
}
}
}
}
"A ByteStringIterator" must {
"behave like a buffered Vector Iterator" when {
"concatenating" in { check { (a: ByteString, b: ByteString) likeVecIts(a, b) { (a, b) (a ++ b).toSeq } } }
"calling head" in { check { a: ByteString a.isEmpty || likeVecIt(a) { _.head } } }
"calling next" in { check { a: ByteString a.isEmpty || likeVecIt(a) { _.next() } } }
"calling hasNext" in { check { a: ByteString likeVecIt(a) { _.hasNext } } }
"calling length" in { check { a: ByteString likeVecIt(a) { _.length } } }
"calling duplicate" in { check { a: ByteString likeVecIt(a)({ _.duplicate match { case (a, b) (a.toSeq, b.toSeq) } }, strict = false) } }
// Have to used toList instead of toSeq here, iterator.span (new in
// Scala-2.9) seems to be broken in combination with toSeq for the
// scala.collection default Iterator (see Scala issue SI-5838).
"calling span" in { check { (a: ByteString, b: Byte) likeVecIt(a)({ _.span(_ != b) match { case (a, b) (a.toList, b.toList) } }, strict = false) } }
"calling takeWhile" in { check { (a: ByteString, b: Byte) likeVecIt(a)({ _.takeWhile(_ != b).toSeq }, strict = false) } }
"calling dropWhile" in { check { (a: ByteString, b: Byte) likeVecIt(a) { _.dropWhile(_ != b).toSeq } } }
"calling indexWhere" in { check { (a: ByteString, b: Byte) likeVecIt(a) { _.indexWhere(_ == b) } } }
"calling indexOf" in { check { (a: ByteString, b: Byte) likeVecIt(a) { _.indexOf(b) } } }
"calling toSeq" in { check { a: ByteString likeVecIt(a) { _.toSeq } } }
"calling foreach" in { check { a: ByteString likeVecIt(a) { it var acc = 0; it foreach { acc += _ }; acc } } }
"calling foldLeft" in { check { a: ByteString likeVecIt(a) { _.foldLeft(0) { _ + _ } } } }
"calling toArray" in { check { a: ByteString likeVecIt(a) { _.toArray.toSeq } } }
"calling slice" in {
check { slice: ByteStringSlice
slice match {
case (xs, from, until) likeVecIt(xs)({
_.slice(from, until).toSeq
}, strict = false)
}
}
}
"calling take and drop" in {
check { slice: ByteStringSlice
slice match {
case (xs, from, until) likeVecIt(xs)({
_.drop(from).take(until - from).toSeq
}, strict = false)
}
}
}
"calling copyToArray" in {
check { slice: ByteStringSlice
slice match {
case (xs, from, until) likeVecIt(xs)({ it
val array = Array.ofDim[Byte](xs.length)
it.slice(from, until).copyToArray(array, from, until)
array.toSeq
}, strict = false)
}
}
}
}
"function as expected" when {
"getting Bytes, using getByte and getBytes" in {
// mixing getByte and getBytes here for more rigorous testing
check { slice: ByteStringSlice
val (bytes, from, until) = slice
val input = bytes.iterator
val output = Array.ofDim[Byte](bytes.length)
for (i 0 to from - 1) output(i) = input.getByte
input.getBytes(output, from, until - from)
for (i until to bytes.length - 1) output(i) = input.getByte
(output.toSeq == bytes) && (input.isEmpty)
}
}
"getting Bytes, using the InputStream wrapper" in {
// combining skip and both read methods here for more rigorous testing
check { slice: ByteStringSlice
val (bytes, from, until) = slice
val a = (0 max from) min bytes.length
val b = (a max until) min bytes.length
val input = bytes.iterator
val output = Array.ofDim[Byte](bytes.length)
input.asInputStream.skip(a)
val toRead = b - a
var (nRead, eof) = (0, false)
while ((nRead < toRead) && !eof) {
val n = input.asInputStream.read(output, a + nRead, toRead - nRead)
if (n == -1) eof = true
else nRead += n
}
if (eof) throw new RuntimeException("Unexpected EOF")
for (i b to bytes.length - 1) output(i) = input.asInputStream.read().toByte
(output.toSeq.drop(a) == bytes.drop(a)) &&
(input.asInputStream.read() == -1) &&
((output.length < 1) || (input.asInputStream.read(output, 0, 1) == -1))
}
}
"calling copyToBuffer" in {
check { bytes: ByteString
import java.nio.ByteBuffer
val buffer = ByteBuffer.allocate(bytes.size)
bytes.copyToBuffer(buffer)
buffer.flip()
val array = Array.ofDim[Byte](bytes.size)
buffer.get(array)
bytes == array.toSeq
}
}
}
"decode data correctly" when {
"decoding Short in big-endian" in { check { slice: ByteStringSlice testShortDecoding(slice, BIG_ENDIAN) } }
"decoding Short in little-endian" in { check { slice: ByteStringSlice testShortDecoding(slice, LITTLE_ENDIAN) } }
"decoding Int in big-endian" in { check { slice: ByteStringSlice testIntDecoding(slice, BIG_ENDIAN) } }
"decoding Int in little-endian" in { check { slice: ByteStringSlice testIntDecoding(slice, LITTLE_ENDIAN) } }
"decoding Long in big-endian" in { check { slice: ByteStringSlice testLongDecoding(slice, BIG_ENDIAN) } }
"decoding Long in little-endian" in { check { slice: ByteStringSlice testLongDecoding(slice, LITTLE_ENDIAN) } }
"decoding Float in big-endian" in { check { slice: ByteStringSlice testFloatDecoding(slice, BIG_ENDIAN) } }
"decoding Float in little-endian" in { check { slice: ByteStringSlice testFloatDecoding(slice, LITTLE_ENDIAN) } }
"decoding Double in big-endian" in { check { slice: ByteStringSlice testDoubleDecoding(slice, BIG_ENDIAN) } }
"decoding Double in little-endian" in { check { slice: ByteStringSlice testDoubleDecoding(slice, LITTLE_ENDIAN) } }
}
}
"A ByteStringBuilder" must {
"function like a VectorBuilder" when {
"adding various contents using ++= and +=" in {
check { (array1: Array[Byte], array2: Array[Byte], bs1: ByteString, bs2: ByteString, bs3: ByteString)
likeVecBld { builder
builder ++= array1
bs1 foreach { b builder += b }
builder ++= bs2
bs3 foreach { b builder += b }
builder ++= Vector(array2: _*)
}
}
}
}
"function as expected" when {
"putting Bytes, using putByte and putBytes" in {
// mixing putByte and putBytes here for more rigorous testing
check { slice: ArraySlice[Byte]
val (data, from, until) = slice
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.putByte(data(i))
builder.putBytes(data, from, until - from)
for (i until to data.length - 1) builder.putByte(data(i))
data.toSeq == builder.result
}
}
"putting Bytes, using the OutputStream wrapper" in {
// mixing the write methods here for more rigorous testing
check { slice: ArraySlice[Byte]
val (data, from, until) = slice
val builder = ByteString.newBuilder
for (i 0 to from - 1) builder.asOutputStream.write(data(i).toInt)
builder.asOutputStream.write(data, from, until - from)
for (i until to data.length - 1) builder.asOutputStream.write(data(i).toInt)
data.toSeq == builder.result
}
}
}
"encode data correctly" when {
"encoding Short in big-endian" in { check { slice: ArraySlice[Short] testShortEncoding(slice, BIG_ENDIAN) } }
"encoding Short in little-endian" in { check { slice: ArraySlice[Short] testShortEncoding(slice, LITTLE_ENDIAN) } }
"encoding Int in big-endian" in { check { slice: ArraySlice[Int] testIntEncoding(slice, BIG_ENDIAN) } }
"encoding Int in little-endian" in { check { slice: ArraySlice[Int] testIntEncoding(slice, LITTLE_ENDIAN) } }
"encoding Long in big-endian" in { check { slice: ArraySlice[Long] testLongEncoding(slice, BIG_ENDIAN) } }
"encoding Long in little-endian" in { check { slice: ArraySlice[Long] testLongEncoding(slice, LITTLE_ENDIAN) } }
"encoding Float in big-endian" in { check { slice: ArraySlice[Float] testFloatEncoding(slice, BIG_ENDIAN) } }
"encoding Float in little-endian" in { check { slice: ArraySlice[Float] testFloatEncoding(slice, LITTLE_ENDIAN) } }
"encoding Double in big-endian" in { check { slice: ArraySlice[Double] testDoubleEncoding(slice, BIG_ENDIAN) } }
"encoding Double in little-endian" in { check { slice: ArraySlice[Double] testDoubleEncoding(slice, LITTLE_ENDIAN) } }
}
}
}

View file

@ -8,10 +8,14 @@ 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,11 @@
package akka.japi;
import scala.collection.Seq;
public class JAPI {
public static <T> Seq<T> seq(T... ts) {
return Util.arrayToSeq(ts);
}
}

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)(@BeanProperty val existenceConfirmed: Boolean)
case class Terminated(@BeanProperty actor: ActorRef)(@BeanProperty val existenceConfirmed: Boolean) extends AutoReceivedMessage
abstract class ReceiveTimeout extends PossiblyHarmful
@ -142,7 +141,7 @@ class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, val
}
/**
* A PostRestartException is thrown when constructor or postRestart() method
* A PostRestartException is thrown when constructor or postRestart() method
* fails during a restart attempt.
*
* @param actor is the actor whose constructor or postRestart() hook failed
@ -159,8 +158,7 @@ class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, val
* 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)
}
@ -303,18 +301,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

@ -13,8 +13,9 @@ import akka.japi.Procedure
import java.io.{ NotSerializableException, ObjectOutputStream }
import akka.serialization.SerializationExtension
import akka.event.Logging.{ LogEventException, LogEvent }
import collection.immutable.{ TreeSet, Stack, TreeMap }
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
@ -167,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(inResponseToFailure: Boolean): 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: 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 {
@ -182,131 +255,12 @@ private[akka] object ActorCell {
def cancel() {}
}
final val emptyReceiveTimeoutData: (Long, Cancellable) = (-1, emptyCancellable)
final val emptyReceiveTimeoutData: (Duration, Cancellable) = (Duration.Undefined, 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
case class Recreation(cause: Throwable) extends SuspendReason
case object Termination extends SuspendReason
trait ChildrenContainer {
def add(child: ActorRef): ChildrenContainer
def remove(child: ActorRef): ChildrenContainer
def getByName(name: String): Option[ChildRestartStats]
def getByRef(actor: ActorRef): Option[ChildRestartStats]
def children: Iterable[ActorRef]
def stats: Iterable[ChildRestartStats]
def shallDie(actor: ActorRef): ChildrenContainer
}
trait EmptyChildrenContainer extends ChildrenContainer {
val emptyStats = TreeMap.empty[String, ChildRestartStats]
def add(child: ActorRef): ChildrenContainer =
new NormalChildrenContainer(emptyStats.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = this
def getByName(name: String): Option[ChildRestartStats] = None
def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
def children: Iterable[ActorRef] = Nil
def stats: Iterable[ChildRestartStats] = Nil
def shallDie(actor: ActorRef): ChildrenContainer = this
override def toString = "no children"
}
/**
* This is the empty container, shared among all leaf actors.
*/
object EmptyChildrenContainer extends EmptyChildrenContainer
/**
* This is the empty container which is installed after the last child has
* terminated while stopping; it is necessary to distinguish from the normal
* empty state while calling handleChildTerminated() for the last time.
*/
object TerminatedChildrenContainer extends EmptyChildrenContainer {
override def add(child: ActorRef): ChildrenContainer = this
}
/**
* Normal children container: we do have at least one child, but none of our
* children are currently terminating (which is the time period between
* calling context.stop(child) and processing the ChildTerminated() system
* message).
*/
class NormalChildrenContainer(c: TreeMap[String, ChildRestartStats]) extends ChildrenContainer {
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
case _ None
}
def children: Iterable[ActorRef] = c.values.view.map(_.child)
def stats: Iterable[ChildRestartStats] = c.values
def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
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 =
if (c.isEmpty) EmptyChildrenContainer
else new NormalChildrenContainer(c)
}
/**
* Waiting state: there are outstanding termination requests (i.e. context.stop(child)
* was called but the corresponding ChildTerminated() system message has not yet been
* processed). There could be no specific reason (UserRequested), we could be Restarting
* or Terminating.
*
* Removing the last child which was supposed to be terminating will return a different
* 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)
extends ChildrenContainer {
def add(child: ActorRef): ChildrenContainer = copy(c.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = {
val t = toDie - child
if (t.isEmpty) reason match {
case Termination TerminatedChildrenContainer
case _ NormalChildrenContainer(c - child.path.name)
}
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
case _ None
}
def children: Iterable[ActorRef] = c.values.view.map(_.child)
def stats: Iterable[ChildRestartStats] = c.values
def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)
override def toString =
if (c.size > 20) c.size + " children"
else c.mkString("children (" + toDie.size + " terminating):\n ", "\n ", "\n") + toDie
}
}
//ACTORCELL IS 64bytes and should stay that way unless very good reason not to (machine sympathy, cache line fit)
@ -316,9 +270,13 @@ private[akka] class ActorCell(
val system: ActorSystemImpl,
val self: InternalActorRef,
val props: Props,
@volatile var parent: InternalActorRef) extends UntypedActorContext {
import AbstractActorCell.mailboxOffset
@volatile var parent: InternalActorRef) extends UntypedActorContext with Cell {
import AbstractActorCell.{ childrenOffset, mailboxOffset, nextNameOffset }
import ActorCell._
import ChildrenContainer._
final def isLocal = true
final def systemImpl = system
@ -328,32 +286,66 @@ private[akka] class ActorCell(
final def provider = system.provider
override final def receiveTimeout: Option[Duration] = if (receiveTimeoutData._1 > 0) Some(Duration(receiveTimeoutData._1, MILLISECONDS)) else None
override final def setReceiveTimeout(timeout: Duration): Unit = setReceiveTimeout(Some(timeout))
final def setReceiveTimeout(timeout: Option[Duration]): Unit = {
val timeoutMs = timeout match {
case None -1L
case Some(duration)
val ms = duration.toMillis
if (ms <= 0) -1L
// 1 millisecond is minimum supported
else if (ms < 1) 1L
else ms
}
receiveTimeoutData = (timeoutMs, receiveTimeoutData._2)
override final def receiveTimeout: Option[Duration] = receiveTimeoutData._1 match {
case Duration.Undefined None
case duration Some(duration)
}
final def setReceiveTimeout(timeout: Option[Duration]): Unit = setReceiveTimeout(timeout.getOrElse(Duration.Undefined))
override final def setReceiveTimeout(timeout: Duration): Unit =
receiveTimeoutData = (
if (Duration.Undefined == timeout || timeout.toMillis < 1) Duration.Undefined else timeout,
receiveTimeoutData._2)
final override def resetReceiveTimeout(): Unit = setReceiveTimeout(None)
/**
* In milliseconds
*/
var receiveTimeoutData: (Long, Cancellable) = emptyReceiveTimeoutData
var receiveTimeoutData: (Duration, 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 +358,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,41 +369,60 @@ 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 = _
var actor: Actor = _
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
@volatile var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
var nextNameSequence: Long = 0
private var behaviorStack: List[Actor.Receive] = emptyBehaviorStack
var watching: Set[ActorRef] = emptyActorRefSet
var watchedBy: Set[ActorRef] = emptyActorRefSet
@ -426,15 +437,17 @@ private[akka] class ActorCell(
def setFailed(): Unit = _failed = true
def setNotFailed(): Unit = _failed = false
//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
@ -454,6 +467,12 @@ private[akka] class ActorCell(
else oldMailbox
}
final def hasMessages: Boolean = mailbox.hasMessages
final def numberOfMessages: Int = mailbox.numberOfMessages
val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
/**
* UntypedActorContext impl
*/
@ -461,20 +480,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.
*/
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 ⬅⬅⬅
@ -529,8 +550,10 @@ private[akka] class ActorCell(
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
@ -540,25 +563,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
}
}
@ -602,7 +621,7 @@ private[akka] class ActorCell(
assert(mailbox.isSuspended, "mailbox must be suspended during restart, status=" + mailbox.status)
childrenRefs match {
case ct: TerminatingChildrenContainer
childrenRefs = ct.copy(reason = Recreation(cause))
setChildrenTerminationReason(Recreation(cause))
case _
doRecreate(cause, failedActor)
}
@ -666,7 +685,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
suspendNonRecursive()
if (system.settings.DebugLifecycle) publish(Debug(self.path.toString, clazz(actor), "stopping"))
@ -675,7 +694,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) publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
}
@ -690,7 +710,7 @@ private[akka] class ActorCell(
case Terminate() terminate()
case Supervise(child) supervise(child)
case ChildTerminated(child) handleChildTerminated(child)
case NoMessage // to shut up the exhaustiveness warning
case NoMessage // only here to suppress warning
}
} catch {
case e @ (_: InterruptedException | NonFatal(_)) handleInvokeFailure(e, "error while processing " + message)
@ -739,10 +759,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
@ -757,8 +775,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 = {
@ -767,6 +786,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)
@ -811,7 +831,7 @@ private[akka] class ActorCell(
finally {
if (system.settings.DebugLifecycle)
publish(Debug(self.path.toString, clazz(a), "stopped"))
behaviorStack = behaviorStackPlaceHolder
behaviorStack = emptyBehaviorStack
clearActorFields(a)
actor = null
}
@ -851,8 +871,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"
@ -860,22 +879,27 @@ private[akka] class ActorCell(
case _
}
case _
childrenRefs = childrenRefs.remove(child)
removeChild(child)
actor.supervisorStrategy.handleChildTerminated(this, child, children)
}
} catch {
case NonFatal(e) handleInvokeFailure(e, "handleChildTerminated failed")
}
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))
final def checkReceiveTimeout() {
val recvtimeout = receiveTimeoutData
if (recvtimeout._1 > 0 && !mailbox.hasMessages) {
if (Duration.Undefined != recvtimeout._1 && !mailbox.hasMessages) {
recvtimeout._2.cancel() //Cancel any ongoing future
//Only reschedule if desired and there are currently no more messages to be processed
receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(Duration(recvtimeout._1, TimeUnit.MILLISECONDS), self, ReceiveTimeout))
receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(recvtimeout._1, self, ReceiveTimeout))
} else cancelReceiveTimeout()
}

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.
*/
@ -228,21 +252,21 @@ private[akka] class LocalActorRef private[akka] (
_props: Props,
_supervisor: InternalActorRef,
override val path: ActorPath)
extends InternalActorRef with LocalRef {
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)
private val actorCell: ActorCell = newActorCell(_system, this, _props, _supervisor)
actorCell.start()
// ➡➡➡ 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)
@ -313,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)

View file

@ -26,12 +26,12 @@ 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.
@ -104,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
@ -481,11 +482,10 @@ class LocalActorRefProvider(
}
}
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)
@ -539,22 +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) // 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
}
private[akka] class GuardianCell(_system: ActorSystemImpl, _self: InternalActorRef, _props: Props, _parent: InternalActorRef)
extends ActorCell(_system, _self, _props, _parent) {
}

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
@ -56,11 +55,15 @@ object ActorSystem {
* obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader,
* then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader
* associated with the ActorSystem class.
*
* @see <a href="http://typesafehub.github.com/config/v0.4.1/" target="_blank">The Typesafe Config Library API Documentation</a>
*/
def create(name: String, config: Config): ActorSystem = apply(name, config)
/**
* Creates a new ActorSystem with the name "default", the specified Config, and specified ClassLoader
*
* @see <a href="http://typesafehub.github.com/config/v0.4.1/" target="_blank">The Typesafe Config Library API Documentation</a>
*/
def create(name: String, config: Config, classLoader: ClassLoader): ActorSystem = apply(name, config, classLoader)
@ -90,11 +93,15 @@ object ActorSystem {
* obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader,
* then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader
* associated with the ActorSystem class.
*
* @see <a href="http://typesafehub.github.com/config/v0.4.1/" target="_blank">The Typesafe Config Library API Documentation</a>
*/
def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader())
/**
* Creates a new ActorSystem with the name "default", the specified Config, and specified ClassLoader
*
* @see <a href="http://typesafehub.github.com/config/v0.4.1/" target="_blank">The Typesafe Config Library API Documentation</a>
*/
def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new ActorSystemImpl(name, config, classLoader).start()
@ -102,11 +109,15 @@ object ActorSystem {
* Settings are the overall ActorSystem Settings which also provides a convenient access to the Config object.
*
* For more detailed information about the different possible configuration options, look in the Akka Documentation under "Configuration"
*
* @see <a href="http://typesafehub.github.com/config/v0.4.1/" target="_blank">The Typesafe Config Library API Documentation</a>
*/
class Settings(classLoader: ClassLoader, cfg: Config, final val name: String) {
/**
* The backing Config of this ActorSystem's Settings
*
* @see <a href="http://typesafehub.github.com/config/v0.4.1/" target="_blank">The Typesafe Config Library API Documentation</a>
*/
final val config: Config = {
val config = cfg.withFallback(ConfigFactory.defaultReference(classLoader))
@ -423,6 +434,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 {
@ -430,7 +448,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._
@ -480,20 +498,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)).mapTo[ActorRef], timeout.duration)
}
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)).mapTo[ActorRef], timeout.duration)
}
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)).mapTo[ActorRef], timeout.duration)
}
def actorOf(props: Props): ActorRef = guardian.underlying.attachChild(props)
def stop(actor: ActorRef): Unit = {
implicit val timeout = settings.CreationTimeout
@ -540,10 +549,10 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
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)
@ -558,8 +567,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 guardian: LocalActorRef = provider.guardian
def systemGuardian: LocalActorRef = provider.systemGuardian
def /(actorName: String): ActorPath = guardian.path / actorName
def /(path: Iterable[String]): ActorPath = guardian.path / path
@ -683,10 +692,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 ChildrenContainer.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)
@ -695,17 +729,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

@ -0,0 +1,189 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import scala.collection.immutable.TreeMap
/**
* INTERNAL API
*/
private[akka] trait ChildrenContainer {
def add(child: ActorRef): ChildrenContainer
def remove(child: ActorRef): ChildrenContainer
def getByName(name: String): Option[ChildRestartStats]
def getByRef(actor: ActorRef): Option[ChildRestartStats]
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
def isTerminating: Boolean = false
def isNormal: Boolean = true
}
/**
* INTERNAL API
*
* This object holds the classes performing the logic of managing the children
* of an actor, hence they are intimately tied to ActorCell.
*/
private[akka] object ChildrenContainer {
sealed trait SuspendReason
case object UserRequest extends SuspendReason
case class Recreation(cause: Throwable) extends SuspendReason
case object Termination extends SuspendReason
trait EmptyChildrenContainer extends ChildrenContainer {
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
def getByName(name: String): Option[ChildRestartStats] = None
def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
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"
}
/**
* This is the empty container, shared among all leaf actors.
*/
object EmptyChildrenContainer extends EmptyChildrenContainer
/**
* This is the empty container which is installed after the last child has
* terminated while stopping; it is necessary to distinguish from the normal
* empty state while calling handleChildTerminated() for the last time.
*/
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")
override def isTerminating: Boolean = true
override def isNormal: Boolean = false
}
/**
* Normal children container: we do have at least one child, but none of our
* children are currently terminating (which is the time period between
* calling context.stop(child) and processing the ChildTerminated() system
* message).
*/
class NormalChildrenContainer(c: TreeMap[String, ChildStats]) extends ChildrenContainer {
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) match {
case s @ Some(_: ChildRestartStats) s.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
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 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, ChildStats]): ChildrenContainer =
if (c.isEmpty) EmptyChildrenContainer
else new NormalChildrenContainer(c)
}
/**
* Waiting state: there are outstanding termination requests (i.e. context.stop(child)
* was called but the corresponding ChildTerminated() system message has not yet been
* processed). There could be no specific reason (UserRequested), we could be Restarting
* or Terminating.
*
* Removing the last child which was supposed to be terminating will return a different
* 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, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
extends ChildrenContainer {
def add(child: ActorRef): ChildrenContainer = copy(c.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = {
val t = toDie - child
if (t.isEmpty) reason match {
case Termination TerminatedChildrenContainer
case _ NormalChildrenContainer(c - child.path.name)
}
else copy(c - child.path.name, t)
}
def getByName(name: String): Option[ChildRestartStats] = c.get(name) match {
case s @ Some(_: ChildRestartStats) s.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
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 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 isTerminating: Boolean = reason == Termination
override def isNormal: Boolean = reason == UserRequest
override def toString =
if (c.size > 20) c.size + " children"
else c.mkString("children (" + toDie.size + " terminating):\n ", "\n ", "\n") + toDie
}
}

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,217 @@
/**
* 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(inResponseToFailure: Boolean): Unit = underlying.resume(inResponseToFailure)
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()
var suspendCount = 0
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 try
for (_ 1 to suspendCount) cell.suspend()
finally
lock.unlock()
}
def system: ActorSystem = systemImpl
def suspend(): Unit = { lock.lock(); suspendCount += 1; lock.unlock() }
def resume(inResponseToFailure: Boolean): Unit = { lock.lock(); suspendCount -= 1; lock.unlock() }
def restart(cause: Throwable): Unit = { lock.lock(); suspendCount -= 1; lock.unlock() }
def stop(): Unit = sendSystemMessage(Terminate())
def isTerminated: Boolean = false
def parent: InternalActorRef = supervisor
def childrenRefs: ChildrenContainer = ChildrenContainer.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)
}
}
@ -228,8 +231,8 @@ 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 _ " " + a.getClass
case r: ActorRefWithCell " " + r.underlying.numberOfMessages + " messages"
case _ " " + a.getClass
}
val parent = a match {
case i: InternalActorRef ", parent: " + i.getParent
@ -265,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

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,24 +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 = actor.systemImpl.deadLetterMailbox
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
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,6 +6,7 @@ package akka.dispatch
import akka.AkkaException
import java.util.{ Comparator, PriorityQueue, Queue, Deque }
import akka.util._
import akka.actor.{ ActorCell, ActorRef, Cell }
import java.util.concurrent._
import annotation.tailrec
import akka.event.Logging.Error
@ -45,11 +46,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.
*/
@ -236,11 +258,12 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
* 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 actor.systemImpl.deadLetterMailbox.systemEnqueue(actor.self, msg)
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))
@ -250,9 +273,6 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage)
}
@inline
final def dispatcher: MessageDispatcher = actor.dispatcher
/**
* Overridable callback to clean up the mailbox,
* called when an actor is unregistered.
@ -271,7 +291,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
}
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)
}
}
@ -309,7 +329,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
}
/**
@ -337,10 +357,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) actor.system.deadLetterMailbox.systemEnqueue(receiver, message)
else {
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
@ -372,11 +393,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
}
}
@ -451,10 +472,20 @@ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
}
/**
* 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
}
/**
@ -464,7 +495,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
}
@ -481,7 +512,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
@ -494,7 +525,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
}
@ -509,7 +540,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
@ -523,7 +554,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
}
@ -540,7 +571,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

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

@ -586,6 +586,7 @@ object Logging {
/** Null Object used for errors without cause Throwable */
object NoCause extends NoStackTrace
}
def noCause = Error.NoCause
/**
* For WARNING Logging
@ -875,3 +876,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

@ -5,6 +5,7 @@
package akka.japi
import scala.Some
import scala.util.control.NoStackTrace
/**
* A Function interface. Used to create first-class-functions is Java.
@ -44,6 +45,76 @@ trait Creator[T] {
def create(): T
}
object PurePartialFunction {
sealed abstract class NoMatchException extends RuntimeException with NoStackTrace
case object NoMatch extends NoMatchException
final def noMatch(): RuntimeException = NoMatch
}
/**
* Helper for implementing a *pure* partial function: it will possibly be
* invoked multiple times for a single application, because its only abstract
* method is used for both isDefinedAt() and apply(); the former is mapped to
* `isCheck == true` and the latter to `isCheck == false` for those cases where
* this is important to know.
*
* Failure to match is signaled by throwing `noMatch()`, i.e. not returning
* normally (the exception used in this case is pre-allocated, hence not
* <i>that</i> expensive).
*
* {{{
* new PurePartialFunction<Object, String>() {
* public String apply(Object in, boolean isCheck) {
* if (in instanceof TheThing) {
* if (isCheck) return null; // to spare the expensive or side-effecting code
* return doSomethingWithTheThing((TheThing) in);
* } else {
* throw noMatch();
* }
* }
* }
* }}}
*
* The typical use of partial functions from Akka looks like the following:
*
* {{{
* if (pf.isDefinedAt(x)) {
* pf.apply(x);
* }
* }}}
*
* i.e. it will first call `PurePartialFunction.apply(x, true)` and if that
* does not throw `noMatch()` it will continue with calling
* `PurePartialFunction.apply(x, false)`.
*/
abstract class PurePartialFunction[A, B] extends scala.runtime.AbstractFunction1[A, B] with PartialFunction[A, B] {
import PurePartialFunction._
def apply(x: A, isCheck: Boolean): B
final def isDefinedAt(x: A): Boolean = try { apply(x, true); true } catch { case NoMatch false }
final def apply(x: A): B = try apply(x, false) catch { case NoMatch throw new MatchError }
}
/**
* This is a specialized variant of PartialFunction which is <b><i>only
* applicable if you know that `isDefinedAt(x)` is always called before
* `apply(x)`with the same `x` of course.</i></b>
*
* `match(x)` will be called for `isDefinedAt(x)` only, and its semantics
* are the same as for [[akka.japi.PurePartialFunction]] (apart from the
* missing because unneeded boolean argument).
*/
abstract class CachingPartialFunction[A, B <: AnyRef] extends scala.runtime.AbstractFunction1[A, B] with PartialFunction[A, B] {
import PurePartialFunction._
def `match`(x: A): B
var cache: B = _
final def isDefinedAt(x: A): Boolean = try { cache = `match`(x); true } catch { case NoMatch cache = null.asInstanceOf[B]; false }
final def apply(x: A): B = cache
}
/**
* This class represents optional values. Instances of <code>Option</code>
* are either instances of case class <code>Some</code> or it is case
@ -117,4 +188,8 @@ object Util {
* Given a Class returns a Scala Manifest of that Class
*/
def manifest[T](clazz: Class[T]): Manifest[T] = Manifest.classType(clazz)
def arrayToSeq[T](arr: Array[T]): Seq[T] = arr.toSeq
def arrayToSeq(classes: Array[Class[_]]): Seq[Class[_]] = classes.toSeq
}

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) {
/*
* 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.
*
* ===> I M P O R T A N T N O T I C E <===
*
* DO NOT THROW ANY EXCEPTIONS BEFORE THE FOLLOWING TRY-BLOCK WITHOUT
* EXITING THE MONITOR OF THE actorCell!
*
* This is important, just dont do it! No kidding.
*/
override def newActorCell(
system: ActorSystemImpl,
ref: InternalActorRef,
props: Props,
supervisor: InternalActorRef): ActorCell = {
val cell = super.newActorCell(system, ref, props, supervisor)
Unsafe.instance.monitorEnter(cell)
cell
// verify that a BalancingDispatcher is not used with a Router
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) {
throw new ConfigurationException(
"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,39 +58,28 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
private var _routeeProvider: RouteeProvider = _
def routeeProvider = _routeeProvider
val route =
try {
// verify that a BalancingDispatcher is not used with a Router
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) {
actorContext.stop(actorContext.self)
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.")
}
_routeeProvider = routerConfig.createRouteeProvider(actorContext)
val r = routerConfig.createRoute(routeeProps, routeeProvider)
// initial resize, before message send
routerConfig.resizer foreach { r
if (r.isTimeForResize(resizeCounter.getAndIncrement()))
r.resize(routeeProps, routeeProvider)
}
r
} finally {
assert(Thread.holdsLock(actorContext))
Unsafe.instance.monitorExit(actorContext) // unblock Routers constructor
val route = {
_routeeProvider = routerConfig.createRouteeProvider(this)
val r = routerConfig.createRoute(routeeProps, routeeProvider)
// initial resize, before message send
routerConfig.resizer foreach { r
if (r.isTimeForResize(resizeCounter.getAndIncrement()))
r.resize(routeeProps, routeeProvider)
}
r
}
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
@ -122,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
}
/**
@ -133,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
@ -147,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)
}
}
}
@ -212,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 = {}
}
/**
@ -227,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.
@ -247,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 {
@ -264,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]
}
/**
@ -298,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 = ({
@ -417,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
}
@ -774,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
}
@ -788,8 +772,8 @@ 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 _ false
case x: ActorRefWithCell x.underlying.hasMessages
case _ false
}
/**
@ -799,8 +783,12 @@ 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 _ false
case x: ActorRefWithCell
x.underlying match {
case cell: ActorCell cell.mailbox.isSuspended
case _ true
}
case _ false
}
/**
@ -810,8 +798,8 @@ 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 _ 0
case x: ActorRefWithCell x.underlying.numberOfMessages
case _ 0
}
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = {
@ -1283,12 +1271,20 @@ case class DefaultResizer(
*/
def pressure(routees: IndexedSeq[ActorRef]): Int = {
routees count {
case a: LocalActorRef
val cell = a.underlying
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 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

@ -0,0 +1,610 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
import java.nio.{ ByteBuffer, ByteOrder }
import scala.collection.{ LinearSeq, IndexedSeqOptimized }
import scala.collection.mutable.{ Builder, WrappedArray }
import scala.collection.immutable.{ IndexedSeq, VectorBuilder }
import scala.collection.generic.CanBuildFrom
import scala.collection.mutable.{ ListBuffer }
import scala.annotation.tailrec
import java.nio.ByteBuffer
object ByteIterator {
object ByteArrayIterator {
private val emptyArray: Array[Byte] = Array.ofDim[Byte](0)
protected[akka] def apply(array: Array[Byte]): ByteArrayIterator =
new ByteArrayIterator(array, 0, array.length)
protected[akka] def apply(array: Array[Byte], from: Int, until: Int): ByteArrayIterator =
new ByteArrayIterator(array, from, until)
val empty: ByteArrayIterator = apply(emptyArray)
}
class ByteArrayIterator private (private var array: Array[Byte], private var from: Int, private var until: Int) extends ByteIterator {
iterator
@inline final def len: Int = until - from
@inline final def hasNext: Boolean = from < until
@inline final def head: Byte = array(from)
final def next(): Byte = {
if (!hasNext) Iterator.empty.next
else { val i = from; from = from + 1; array(i) }
}
def clear(): Unit = { this.array = ByteArrayIterator.emptyArray; from = 0; until = from }
final override def length: Int = { val l = len; clear(); l }
final override def ++(that: TraversableOnce[Byte]): ByteIterator = that match {
case that: ByteIterator
if (that.isEmpty) this
else if (this.isEmpty) that
else that match {
case that: ByteArrayIterator
if ((this.array eq that.array) && (this.until == that.from)) {
this.until = that.until
that.clear()
this
} else {
val result = MultiByteArrayIterator(List(this, that))
this.clear()
result
}
case that: MultiByteArrayIterator this ++: that
}
case _ super.++(that)
}
final override def clone: ByteArrayIterator = new ByteArrayIterator(array, from, until)
final override def take(n: Int): this.type = {
if (n < len) until = { if (n > 0) (from + n) else from }
this
}
final override def drop(n: Int): this.type = {
if (n > 0) from = { if (n < len) (from + n) else until }
this
}
final override def takeWhile(p: Byte Boolean): this.type = {
val prev = from
dropWhile(p)
until = from; from = prev
this
}
final override def dropWhile(p: Byte Boolean): this.type = {
var stop = false
while (!stop && hasNext) {
if (p(array(from))) { from = from + 1 } else { stop = true }
}
this
}
final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = {
val n = 0 max ((xs.length - start) min this.len min len)
Array.copy(this.array, from, xs, start, n)
this.drop(n)
}
final override def toByteString: ByteString = {
val result =
if ((from == 0) && (until == array.length)) ByteString.ByteString1C(array)
else ByteString.ByteString1(array, from, len)
clear()
result
}
def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type = {
if (n <= this.len) {
Array.copy(this.array, this.from, xs, offset, n)
this.drop(n)
} else Iterator.empty.next
}
private def wrappedByteBuffer: ByteBuffer = ByteBuffer.wrap(array, from, len).asReadOnlyBuffer
def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
{ wrappedByteBuffer.order(byteOrder).asShortBuffer.get(xs, offset, n); drop(2 * n) }
def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
{ wrappedByteBuffer.order(byteOrder).asIntBuffer.get(xs, offset, n); drop(4 * n) }
def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
{ wrappedByteBuffer.order(byteOrder).asLongBuffer.get(xs, offset, n); drop(8 * n) }
def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
{ wrappedByteBuffer.order(byteOrder).asFloatBuffer.get(xs, offset, n); drop(4 * n) }
def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
{ wrappedByteBuffer.order(byteOrder).asDoubleBuffer.get(xs, offset, n); drop(8 * n) }
def copyToBuffer(buffer: ByteBuffer): Int = {
val copyLength = math.min(buffer.remaining, len)
if (copyLength > 0) {
buffer.put(array, from, copyLength)
drop(copyLength)
}
copyLength
}
def asInputStream: java.io.InputStream = new java.io.InputStream {
override def available: Int = iterator.len
def read: Int = if (hasNext) (next().toInt & 0xff) else -1
override def read(b: Array[Byte], off: Int, len: Int): Int = {
if ((off < 0) || (len < 0) || (off + len > b.length)) throw new IndexOutOfBoundsException
if (len == 0) 0
else if (!isEmpty) {
val nRead = math.min(available, len)
copyToArray(b, off, nRead)
nRead
} else -1
}
override def skip(n: Long): Long = {
val nSkip = math.min(iterator.len, n.toInt)
iterator.drop(nSkip)
nSkip
}
}
}
object MultiByteArrayIterator {
protected val clearedList: List[ByteArrayIterator] = List(ByteArrayIterator.empty)
val empty: MultiByteArrayIterator = new MultiByteArrayIterator(Nil)
protected[akka] def apply(iterators: LinearSeq[ByteArrayIterator]): MultiByteArrayIterator =
new MultiByteArrayIterator(iterators)
}
class MultiByteArrayIterator private (private var iterators: LinearSeq[ByteArrayIterator]) extends ByteIterator {
// After normalization:
// * iterators.isEmpty == false
// * (!iterator.head.isEmpty || iterators.tail.isEmpty) == true
private def normalize(): this.type = {
@tailrec def norm(xs: LinearSeq[ByteArrayIterator]): LinearSeq[ByteArrayIterator] = {
if (xs.isEmpty) MultiByteArrayIterator.clearedList
else if (xs.head.isEmpty) norm(xs.tail)
else xs
}
iterators = norm(iterators)
this
}
normalize()
@inline private def current: ByteArrayIterator = iterators.head
@inline private def dropCurrent(): Unit = { iterators = iterators.tail }
@inline def clear(): Unit = { iterators = MultiByteArrayIterator.empty.iterators }
@inline final def hasNext: Boolean = current.hasNext
@inline final def head: Byte = current.head
final def next(): Byte = {
val result = current.next()
normalize()
result
}
final override def len: Int = iterators.foldLeft(0) { _ + _.len }
final override def length: Int = {
var result = len
clear()
result
}
private[akka] def ++:(that: ByteArrayIterator): this.type = {
iterators = that +: iterators
this
}
final override def ++(that: TraversableOnce[Byte]): ByteIterator = that match {
case that: ByteIterator
if (that.isEmpty) this
else if (this.isEmpty) that
else {
that match {
case that: ByteArrayIterator
iterators = this.iterators :+ that
that.clear()
this
case that: MultiByteArrayIterator
iterators = this.iterators ++ that.iterators
that.clear()
this
}
}
case _ super.++(that)
}
final override def clone: MultiByteArrayIterator = {
val clonedIterators: List[ByteArrayIterator] = iterators.map(_.clone)(collection.breakOut)
new MultiByteArrayIterator(clonedIterators)
}
final override def take(n: Int): this.type = {
var rest = n
val builder = new ListBuffer[ByteArrayIterator]
while ((rest > 0) && !iterators.isEmpty) {
current.take(rest)
if (current.hasNext) {
rest -= current.len
builder += current
}
iterators = iterators.tail
}
iterators = builder.result
normalize()
}
@tailrec final override def drop(n: Int): this.type = if ((n > 0) && !isEmpty) {
val nCurrent = math.min(n, current.len)
current.drop(n)
val rest = n - nCurrent
assert(current.isEmpty || (rest == 0))
normalize()
drop(rest)
} else this
final override def takeWhile(p: Byte Boolean): this.type = {
var stop = false
var builder = new ListBuffer[ByteArrayIterator]
while (!stop && !iterators.isEmpty) {
val lastLen = current.len
current.takeWhile(p)
if (current.hasNext) builder += current
if (current.len < lastLen) stop = true
dropCurrent()
}
iterators = builder.result
normalize()
}
@tailrec final override def dropWhile(p: Byte Boolean): this.type = if (!isEmpty) {
current.dropWhile(p)
val dropMore = current.isEmpty
normalize()
if (dropMore) dropWhile(p) else this
} else this
final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = {
var pos = start
var rest = len
while ((rest > 0) && !iterators.isEmpty) {
val n = 0 max ((xs.length - pos) min current.len min rest)
current.copyToArray(xs, pos, n)
pos += n
rest -= n
dropCurrent()
}
normalize()
}
override def foreach[@specialized U](f: Byte U): Unit = {
iterators foreach { _ foreach f }
clear()
}
final override def toByteString: ByteString = {
if (iterators.tail isEmpty) iterators.head.toByteString
else {
val result = iterators.foldLeft(ByteString.empty) { _ ++ _.toByteString }
clear()
result
}
}
@tailrec protected final def getToArray[A](xs: Array[A], offset: Int, n: Int, elemSize: Int)(getSingle: A)(getMult: (Array[A], Int, Int) Unit): this.type = if (n <= 0) this else {
if (isEmpty) Iterator.empty.next
val nDone = if (current.len >= elemSize) {
val nCurrent = math.min(n, current.len / elemSize)
getMult(xs, offset, nCurrent)
nCurrent
} else {
xs(offset) = getSingle
1
}
normalize()
getToArray(xs, offset + nDone, n - nDone, elemSize)(getSingle)(getMult)
}
def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type =
getToArray(xs, offset, n, 1) { getByte } { current.getBytes(_, _, _) }
def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 2) { getShort(byteOrder) } { current.getShorts(_, _, _)(byteOrder) }
def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 4) { getInt(byteOrder) } { current.getInts(_, _, _)(byteOrder) }
def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 8) { getLong(byteOrder) } { current.getLongs(_, _, _)(byteOrder) }
def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 8) { getFloat(byteOrder) } { current.getFloats(_, _, _)(byteOrder) }
def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type =
getToArray(xs, offset, n, 8) { getDouble(byteOrder) } { current.getDoubles(_, _, _)(byteOrder) }
def copyToBuffer(buffer: ByteBuffer): Int = {
val n = iterators.foldLeft(0) { _ + _.copyToBuffer(buffer) }
normalize()
n
}
def asInputStream: java.io.InputStream = new java.io.InputStream {
override def available: Int = current.len
def read: Int = if (hasNext) (next().toInt & 0xff) else -1
override def read(b: Array[Byte], off: Int, len: Int): Int = {
val nRead = current.asInputStream.read(b, off, len)
normalize()
nRead
}
override def skip(n: Long): Long = {
@tailrec def skipImpl(n: Long, skipped: Long): Long = if (n > 0) {
if (!isEmpty) {
val m = current.asInputStream.skip(n)
normalize()
val newN = n - m
val newSkipped = skipped + m
if (newN > 0) skipImpl(newN, newSkipped)
else newSkipped
} else 0
} else 0
skipImpl(n, 0)
}
}
}
}
/**
* An iterator over a ByteString.
*/
abstract class ByteIterator extends BufferedIterator[Byte] {
def len: Int
def head: Byte
def next(): Byte
protected def clear(): Unit
def ++(that: TraversableOnce[Byte]): ByteIterator = if (that.isEmpty) this else ByteIterator.ByteArrayIterator(that.toArray)
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def clone: ByteIterator = throw new UnsupportedOperationException("Method clone is not implemented in ByteIterator")
override def duplicate: (ByteIterator, ByteIterator) = (this, clone)
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def take(n: Int): this.type = throw new UnsupportedOperationException("Method take is not implemented in ByteIterator")
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def drop(n: Int): this.type = throw new UnsupportedOperationException("Method drop is not implemented in ByteIterator")
override def slice(from: Int, until: Int): this.type = {
if (from > 0) drop(from).take(until - from)
else take(until)
}
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def takeWhile(p: Byte Boolean): this.type = throw new UnsupportedOperationException("Method takeWhile is not implemented in ByteIterator")
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// the parent class.
override def dropWhile(p: Byte Boolean): this.type = throw new UnsupportedOperationException("Method dropWhile is not implemented in ByteIterator")
override def span(p: Byte Boolean): (ByteIterator, ByteIterator) = {
val that = clone
this.takeWhile(p)
that.drop(this.len)
(this, that)
}
override def indexWhere(p: Byte Boolean): Int = {
var index = 0
var found = false
while (!found && hasNext) if (p(next())) { found = true } else { index += 1 }
if (found) index else -1
}
def indexOf(elem: Byte): Int = indexWhere { _ == elem }
override def indexOf[B >: Byte](elem: B): Int = indexWhere { _ == elem }
def toByteString: ByteString
override def toSeq: ByteString = toByteString
override def foreach[@specialized U](f: Byte U): Unit =
while (hasNext) f(next())
override def foldLeft[@specialized B](z: B)(op: (B, Byte) B): B = {
var acc = z
foreach { byte acc = op(acc, byte) }
acc
}
override def toArray[B >: Byte](implicit arg0: ClassManifest[B]): Array[B] = {
val target = Array.ofDim[B](len)
copyToArray(target)
target
}
/**
* Get a single Byte from this iterator. Identical to next().
*/
def getByte: Byte = next()
/**
* Get a single Short from this iterator.
*/
def getShort(implicit byteOrder: ByteOrder): Short = {
if (byteOrder == ByteOrder.BIG_ENDIAN)
((next() & 0xff) << 8 | (next() & 0xff) << 0).toShort
else if (byteOrder == ByteOrder.LITTLE_ENDIAN)
((next() & 0xff) << 0 | (next() & 0xff) << 8).toShort
else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
/**
* Get a single Int from this iterator.
*/
def getInt(implicit byteOrder: ByteOrder): Int = {
if (byteOrder == ByteOrder.BIG_ENDIAN)
((next() & 0xff) << 24
| (next() & 0xff) << 16
| (next() & 0xff) << 8
| (next() & 0xff) << 0)
else if (byteOrder == ByteOrder.LITTLE_ENDIAN)
((next() & 0xff) << 0
| (next() & 0xff) << 8
| (next() & 0xff) << 16
| (next() & 0xff) << 24)
else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
/**
* Get a single Long from this iterator.
*/
def getLong(implicit byteOrder: ByteOrder): Long = {
if (byteOrder == ByteOrder.BIG_ENDIAN)
((next().toLong & 0xff) << 56
| (next().toLong & 0xff) << 48
| (next().toLong & 0xff) << 40
| (next().toLong & 0xff) << 32
| (next().toLong & 0xff) << 24
| (next().toLong & 0xff) << 16
| (next().toLong & 0xff) << 8
| (next().toLong & 0xff) << 0)
else if (byteOrder == ByteOrder.LITTLE_ENDIAN)
((next().toLong & 0xff) << 0
| (next().toLong & 0xff) << 8
| (next().toLong & 0xff) << 16
| (next().toLong & 0xff) << 24
| (next().toLong & 0xff) << 32
| (next().toLong & 0xff) << 40
| (next().toLong & 0xff) << 48
| (next().toLong & 0xff) << 56)
else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
def getFloat(implicit byteOrder: ByteOrder): Float =
java.lang.Float.intBitsToFloat(getInt(byteOrder))
def getDouble(implicit byteOrder: ByteOrder): Double =
java.lang.Double.longBitsToDouble(getLong(byteOrder))
/**
* Get a specific number of Bytes from this iterator. In contrast to
* copyToArray, this method will fail if this.len < xs.length.
*/
def getBytes(xs: Array[Byte]): this.type = getBytes(xs, 0, xs.length)
/**
* Get a specific number of Bytes from this iterator. In contrast to
* copyToArray, this method will fail if length < n or if (xs.length - offset) < n.
*/
def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type
/**
* Get a number of Shorts from this iterator.
*/
def getShorts(xs: Array[Short])(implicit byteOrder: ByteOrder): this.type =
getShorts(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Shorts from this iterator.
*/
def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Ints from this iterator.
*/
def getInts(xs: Array[Int])(implicit byteOrder: ByteOrder): this.type =
getInts(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Ints from this iterator.
*/
def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Longs from this iterator.
*/
def getLongs(xs: Array[Long])(implicit byteOrder: ByteOrder): this.type =
getLongs(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Longs from this iterator.
*/
def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Floats from this iterator.
*/
def getFloats(xs: Array[Float])(implicit byteOrder: ByteOrder): this.type =
getFloats(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Floats from this iterator.
*/
def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Get a number of Doubles from this iterator.
*/
def getDoubles(xs: Array[Double])(implicit byteOrder: ByteOrder): this.type =
getDoubles(xs, 0, xs.length)(byteOrder)
/**
* Get a number of Doubles from this iterator.
*/
def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type
/**
* Copy as many bytes as possible to a ByteBuffer, starting from it's
* current position. This method will not overflow the buffer.
*
* @param buffer a ByteBuffer to copy bytes to
* @return the number of bytes actually copied
*/
def copyToBuffer(buffer: ByteBuffer): Int
/**
* Directly wraps this ByteIterator in an InputStream without copying.
* Read and skip operations on the stream will advance the iterator
* accordingly.
*/
def asInputStream: java.io.InputStream
}

View file

@ -4,14 +4,13 @@
package akka.util
import java.nio.ByteBuffer
import java.nio.{ ByteBuffer, ByteOrder }
import scala.collection.IndexedSeqOptimized
import scala.collection.mutable.{ Builder, WrappedArray }
import scala.collection.immutable.{ IndexedSeq, VectorBuilder }
import scala.collection.generic.CanBuildFrom
//FIXME MORE DOCS
object ByteString {
/**
@ -67,7 +66,7 @@ object ByteString {
}
/**
* A compact (unsliced) and unfragmented ByteString, implementaton of ByteString1C.
* A compact (unsliced) and unfragmented ByteString, implementation of ByteString1C.
*/
@SerialVersionUID(3956956327691936932L)
final class ByteString1C private (private val bytes: Array[Byte]) extends CompactByteString {
@ -75,32 +74,29 @@ object ByteString {
override def length: Int = bytes.length
def toArray: Array[Byte] = bytes.clone
override def iterator: ByteIterator.ByteArrayIterator = ByteIterator.ByteArrayIterator(bytes, 0, bytes.length)
def toByteString1: ByteString1 = ByteString1(bytes)
private[akka] def toByteString1: ByteString1 = ByteString1(bytes)
override def clone: ByteString1C = new ByteString1C(toArray)
def compact: ByteString1C = this
def asByteBuffer: ByteBuffer = toByteString1.asByteBuffer
def asByteBuffer: ByteBuffer =
toByteString1.asByteBuffer
def decodeString(charset: String): String = new String(bytes, charset)
def ++(that: ByteString): ByteString = if (!that.isEmpty) toByteString1 ++ that else this
def ++(that: ByteString): ByteString =
if (that.isEmpty) this
else if (this.isEmpty) that
else toByteString1 ++ that
override def slice(from: Int, until: Int): ByteString =
if ((from != 0) || (until != length)) toByteString1.slice(from, until)
else this
override def copyToArray[A >: Byte](xs: Array[A], start: Int, len: Int): Unit =
toByteString1.copyToArray(xs, start, len)
def copyToBuffer(buffer: ByteBuffer): Int = toByteString1.copyToBuffer(buffer)
}
private[akka] object ByteString1 {
def apply(bytes: Array[Byte]): ByteString1 = new ByteString1(bytes)
def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 =
new ByteString1(bytes, startIndex, length)
}
/**
@ -112,6 +108,9 @@ object ByteString {
def apply(idx: Int): Byte = bytes(checkRangeConvert(idx))
override def iterator: ByteIterator.ByteArrayIterator =
ByteIterator.ByteArrayIterator(bytes, startIndex, startIndex + length)
private def checkRangeConvert(index: Int): Int = {
if (0 <= index && length > index)
index + startIndex
@ -119,15 +118,10 @@ object ByteString {
throw new IndexOutOfBoundsException(index.toString)
}
def toArray: Array[Byte] = {
val ar = new Array[Byte](length)
Array.copy(bytes, startIndex, ar, 0, length)
ar
}
def isCompact: Boolean = (length == bytes.length)
override def clone: CompactByteString = ByteString1C(toArray)
def compact: CompactByteString = if (length == bytes.length) ByteString1C(bytes) else clone
def compact: CompactByteString =
if (isCompact) ByteString1C(bytes) else ByteString1C(toArray)
def asByteBuffer: ByteBuffer = {
val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer
@ -138,26 +132,17 @@ object ByteString {
def decodeString(charset: String): String =
new String(if (length == bytes.length) bytes else toArray, charset)
def ++(that: ByteString): ByteString = that match {
case b: ByteString1C ByteStrings(this, b.toByteString1)
case b: ByteString1 ByteStrings(this, b)
case bs: ByteStrings ByteStrings(this, bs)
}
override def slice(from: Int, until: Int): ByteString = {
val newStartIndex = math.max(from, 0) + startIndex
val newLength = math.min(until, length) - from
if (newLength <= 0) ByteString.empty
else new ByteString1(bytes, newStartIndex, newLength)
}
override def copyToArray[A >: Byte](xs: Array[A], start: Int, len: Int): Unit =
Array.copy(bytes, startIndex, xs, start, math.min(math.min(length, len), xs.length - start))
def copyToBuffer(buffer: ByteBuffer): Int = {
val copyLength = math.min(buffer.remaining, length)
if (copyLength > 0) buffer.put(bytes, startIndex, copyLength)
copyLength
def ++(that: ByteString): ByteString = {
if (that.isEmpty) this
else if (this.isEmpty) that
else that match {
case b: ByteString1C ByteStrings(this, b.toByteString1)
case b: ByteString1
if ((bytes eq b.bytes) && (startIndex + length == b.startIndex))
new ByteString1(bytes, startIndex, length + b.length)
else ByteStrings(this, b)
case bs: ByteStrings ByteStrings(this, bs)
}
}
}
@ -195,7 +180,6 @@ object ByteString {
}
// 0: both empty, 1: 2nd empty, 2: 1st empty, 3: neither empty
// Using length to check emptiness is prohibited by law
def compare(b1: ByteString, b2: ByteString): Int =
if (b1.isEmpty)
if (b2.isEmpty) 0 else 2
@ -206,7 +190,8 @@ object ByteString {
/**
* A ByteString with 2 or more fragments.
*/
final class ByteStrings private (val bytestrings: Vector[ByteString1], val length: Int) extends ByteString {
final class ByteStrings private (private[akka] val bytestrings: Vector[ByteString1], val length: Int) extends ByteString {
if (bytestrings.isEmpty) throw new IllegalArgumentException("bytestrings must not be empty")
def apply(idx: Int): Byte =
if (0 <= idx && idx < length) {
@ -219,71 +204,37 @@ object ByteString {
bytestrings(pos)(idx - seen)
} else throw new IndexOutOfBoundsException(idx.toString)
override def slice(from: Int, until: Int): ByteString = {
val start = math.max(from, 0)
val end = math.min(until, length)
if (end <= start)
ByteString.empty
else {
val iter = bytestrings.iterator
var cur = iter.next
var pos = 0
var seen = 0
while (from >= seen + cur.length) {
seen += cur.length
pos += 1
cur = iter.next
}
val startpos = pos
val startidx = start - seen
while (until > seen + cur.length) {
seen += cur.length
pos += 1
cur = iter.next
}
val endpos = pos
val endidx = end - seen
if (startpos == endpos)
cur.slice(startidx, endidx)
else {
val first = bytestrings(startpos).drop(startidx).asInstanceOf[ByteString1]
val last = cur.take(endidx).asInstanceOf[ByteString1]
if ((endpos - startpos) == 1)
new ByteStrings(Vector(first, last), until - from)
else
new ByteStrings(first +: bytestrings.slice(startpos + 1, endpos) :+ last, until - from)
}
override def iterator: ByteIterator.MultiByteArrayIterator =
ByteIterator.MultiByteArrayIterator(bytestrings.toStream map { _.iterator })
def ++(that: ByteString): ByteString = {
if (that.isEmpty) this
else if (this.isEmpty) that
else that match {
case b: ByteString1C ByteStrings(this, b.toByteString1)
case b: ByteString1 ByteStrings(this, b)
case bs: ByteStrings ByteStrings(this, bs)
}
}
def ++(that: ByteString): ByteString = that match {
case b: ByteString1C ByteStrings(this, b.toByteString1)
case b: ByteString1 ByteStrings(this, b)
case bs: ByteStrings ByteStrings(this, bs)
}
def isCompact: Boolean = if (bytestrings.length == 1) bytestrings.head.isCompact else false
def compact: CompactByteString = {
val ar = new Array[Byte](length)
var pos = 0
bytestrings foreach { b
b.copyToArray(ar, pos, b.length)
pos += b.length
if (isCompact) bytestrings.head.compact
else {
val ar = new Array[Byte](length)
var pos = 0
bytestrings foreach { b
b.copyToArray(ar, pos, b.length)
pos += b.length
}
ByteString1C(ar)
}
ByteString1C(ar)
}
def asByteBuffer: ByteBuffer = compact.asByteBuffer
def decodeString(charset: String): String = compact.decodeString(charset)
def copyToBuffer(buffer: ByteBuffer): Int = {
val copyLength = math.min(buffer.remaining, length)
val iter = bytestrings.iterator
while (iter.hasNext && buffer.hasRemaining) {
iter.next.copyToBuffer(buffer)
}
copyLength
}
}
}
@ -297,7 +248,44 @@ object ByteString {
* TODO: Add performance characteristics
*/
sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimized[Byte, ByteString] {
override protected[this] def newBuilder = ByteString.newBuilder
def apply(idx: Int): Byte
override protected[this] def newBuilder: ByteStringBuilder = ByteString.newBuilder
// *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in
// a parent trait.
override def iterator: ByteIterator = throw new UnsupportedOperationException("Method iterator is not implemented in ByteString")
override def head: Byte = apply(0)
override def tail: ByteString = drop(1)
override def last: Byte = apply(length - 1)
override def init: ByteString = dropRight(1)
override def slice(from: Int, until: Int): ByteString =
if ((from == 0) && (until == length)) this
else iterator.slice(from, until).toByteString
override def take(n: Int): ByteString = slice(0, n)
override def takeRight(n: Int): ByteString = slice(length - n, length)
override def drop(n: Int): ByteString = slice(n, length)
override def dropRight(n: Int): ByteString = slice(0, length - n)
override def takeWhile(p: Byte Boolean): ByteString = iterator.takeWhile(p).toByteString
override def dropWhile(p: Byte Boolean): ByteString = iterator.dropWhile(p).toByteString
override def span(p: Byte Boolean): (ByteString, ByteString) =
{ val (a, b) = iterator.span(p); (a.toByteString, b.toByteString) }
override def splitAt(n: Int): (ByteString, ByteString) = (take(n), drop(n))
override def indexWhere(p: Byte Boolean): Int = iterator.indexWhere(p)
override def indexOf[B >: Byte](elem: B): Int = iterator.indexOf(elem)
override def toArray[B >: Byte](implicit arg0: ClassManifest[B]): Array[B] = iterator.toArray
override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit =
iterator.copyToArray(xs, start, len)
override def foreach[@specialized U](f: Byte U): Unit = iterator foreach f
/**
* Efficiently concatenate another ByteString.
@ -311,14 +299,24 @@ sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimiz
* @param buffer a ByteBuffer to copy bytes to
* @return the number of bytes actually copied
*/
def copyToBuffer(buffer: ByteBuffer): Int
def copyToBuffer(buffer: ByteBuffer): Int = iterator.copyToBuffer(buffer)
/**
* Create a new ByteString with all contents compacted into a single
* byte array.
* Create a new ByteString with all contents compacted into a single,
* full byte array.
* If isCompact returns true, compact is an O(1) operation, but
* might return a different object with an optimized implementation.
*/
def compact: CompactByteString
/**
* Check whether this ByteString is compact in memory.
* If the ByteString is compact, it might, however, not be represented
* by an object that takes full advantage of that fact. Use compact to
* get such an object.
*/
def isCompact: Boolean
/**
* Returns a read-only ByteBuffer that directly wraps this ByteString
* if it is not fragmented.
@ -329,7 +327,7 @@ sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimiz
* Creates a new ByteBuffer with a copy of all bytes contained in this
* ByteString.
*/
final def toByteBuffer: ByteBuffer = ByteBuffer.wrap(toArray)
def toByteBuffer: ByteBuffer = ByteBuffer.wrap(toArray)
/**
* Decodes this ByteString as a UTF-8 encoded String.
@ -420,9 +418,15 @@ object CompactByteString {
}
/**
* A compact, unfragmented ByteString.
* A compact ByteString.
*
* The ByteString is guarantied to be contiguous in memory and to use only
* as much memory as required for its contents.
*/
sealed abstract class CompactByteString extends ByteString with Serializable
sealed abstract class CompactByteString extends ByteString with Serializable {
def isCompact: Boolean = true
def compact: this.type = this
}
/**
* A mutable builder for efficiently creating a [[akka.util.ByteString]].
@ -430,12 +434,37 @@ sealed abstract class CompactByteString extends ByteString with Serializable
* The created ByteString is not automatically compacted.
*/
final class ByteStringBuilder extends Builder[Byte, ByteString] {
builder
import ByteString.{ ByteString1C, ByteString1, ByteStrings }
private var _length = 0
private val _builder = new VectorBuilder[ByteString1]()
private var _length: Int = 0
private val _builder: VectorBuilder[ByteString1] = new VectorBuilder[ByteString1]()
private var _temp: Array[Byte] = _
private var _tempLength = 0
private var _tempCapacity = 0
private var _tempLength: Int = 0
private var _tempCapacity: Int = 0
protected def fillArray(len: Int)(fill: (Array[Byte], Int) Unit): this.type = {
ensureTempSize(_tempLength + len)
fill(_temp, _tempLength)
_tempLength += len
_length += len
this
}
protected def fillByteBuffer(len: Int, byteOrder: ByteOrder)(fill: ByteBuffer Unit): this.type = {
fillArray(len) {
case (array, start)
val buffer = ByteBuffer.wrap(array, start, len)
buffer.order(byteOrder)
fill(buffer)
}
}
def length: Int = _length
override def sizeHint(len: Int): Unit = {
resizeTemp(len - (_length - _tempLength))
}
private def clearTemp(): Unit = {
if (_tempLength > 0) {
@ -487,15 +516,169 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
clearTemp()
_builder += ByteString1(xs.array.clone)
_length += xs.length
case _: collection.IndexedSeq[_]
case seq: collection.IndexedSeq[_]
ensureTempSize(_tempLength + xs.size)
xs.copyToArray(_temp, _tempLength)
_tempLength += seq.length
_length += seq.length
case _
super.++=(xs)
}
this
}
/**
* Add a single Byte to this builder.
*/
def putByte(x: Byte): this.type = this += x
/**
* Add a single Short to this builder.
*/
def putShort(x: Int)(implicit byteOrder: ByteOrder): this.type = {
if (byteOrder == ByteOrder.BIG_ENDIAN) {
this += (x >>> 8).toByte
this += (x >>> 0).toByte
} else if (byteOrder == ByteOrder.LITTLE_ENDIAN) {
this += (x >>> 0).toByte
this += (x >>> 8).toByte
} else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
/**
* Add a single Int to this builder.
*/
def putInt(x: Int)(implicit byteOrder: ByteOrder): this.type = {
fillArray(4) {
case (target, offset)
if (byteOrder == ByteOrder.BIG_ENDIAN) {
target(offset + 0) = (x >>> 24).toByte
target(offset + 1) = (x >>> 16).toByte
target(offset + 2) = (x >>> 8).toByte
target(offset + 3) = (x >>> 0).toByte
} else if (byteOrder == ByteOrder.LITTLE_ENDIAN) {
target(offset + 0) = (x >>> 0).toByte
target(offset + 1) = (x >>> 8).toByte
target(offset + 2) = (x >>> 16).toByte
target(offset + 3) = (x >>> 24).toByte
} else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
this
}
/**
* Add a single Long to this builder.
*/
def putLong(x: Long)(implicit byteOrder: ByteOrder): this.type = {
fillArray(8) {
case (target, offset)
if (byteOrder == ByteOrder.BIG_ENDIAN) {
target(offset + 0) = (x >>> 56).toByte
target(offset + 1) = (x >>> 48).toByte
target(offset + 2) = (x >>> 40).toByte
target(offset + 3) = (x >>> 32).toByte
target(offset + 4) = (x >>> 24).toByte
target(offset + 5) = (x >>> 16).toByte
target(offset + 6) = (x >>> 8).toByte
target(offset + 7) = (x >>> 0).toByte
} else if (byteOrder == ByteOrder.LITTLE_ENDIAN) {
target(offset + 0) = (x >>> 0).toByte
target(offset + 1) = (x >>> 8).toByte
target(offset + 2) = (x >>> 16).toByte
target(offset + 3) = (x >>> 24).toByte
target(offset + 4) = (x >>> 32).toByte
target(offset + 5) = (x >>> 40).toByte
target(offset + 6) = (x >>> 48).toByte
target(offset + 7) = (x >>> 56).toByte
} else throw new IllegalArgumentException("Unknown byte order " + byteOrder)
}
this
}
/**
* Add a single Float to this builder.
*/
def putFloat(x: Float)(implicit byteOrder: ByteOrder): this.type =
putInt(java.lang.Float.floatToRawIntBits(x))(byteOrder)
/**
* Add a single Double to this builder.
*/
def putDouble(x: Double)(implicit byteOrder: ByteOrder): this.type =
putLong(java.lang.Double.doubleToRawLongBits(x))(byteOrder)
/**
* Add a number of Bytes from an array to this builder.
*/
def putBytes(array: Array[Byte]): this.type =
putBytes(array, 0, array.length)
/**
* Add a number of Bytes from an array to this builder.
*/
def putBytes(array: Array[Byte], start: Int, len: Int): this.type =
fillArray(len) { case (target, targetOffset) Array.copy(array, start, target, targetOffset, len) }
/**
* Add a number of Shorts from an array to this builder.
*/
def putShorts(array: Array[Short])(implicit byteOrder: ByteOrder): this.type =
putShorts(array, 0, array.length)(byteOrder)
/**
* Add a number of Shorts from an array to this builder.
*/
def putShorts(array: Array[Short], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type =
fillByteBuffer(len * 2, byteOrder) { _.asShortBuffer.put(array, start, len) }
/**
* Add a number of Ints from an array to this builder.
*/
def putInts(array: Array[Int])(implicit byteOrder: ByteOrder): this.type =
putInts(array, 0, array.length)(byteOrder)
/**
* Add a number of Ints from an array to this builder.
*/
def putInts(array: Array[Int], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type =
fillByteBuffer(len * 4, byteOrder) { _.asIntBuffer.put(array, start, len) }
/**
* Add a number of Longs from an array to this builder.
*/
def putLongs(array: Array[Long])(implicit byteOrder: ByteOrder): this.type =
putLongs(array, 0, array.length)(byteOrder)
/**
* Add a number of Longs from an array to this builder.
*/
def putLongs(array: Array[Long], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type =
fillByteBuffer(len * 8, byteOrder) { _.asLongBuffer.put(array, start, len) }
/**
* Add a number of Floats from an array to this builder.
*/
def putFloats(array: Array[Float])(implicit byteOrder: ByteOrder): this.type =
putFloats(array, 0, array.length)(byteOrder)
/**
* Add a number of Floats from an array to this builder.
*/
def putFloats(array: Array[Float], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type =
fillByteBuffer(len * 4, byteOrder) { _.asFloatBuffer.put(array, start, len) }
/**
* Add a number of Doubles from an array to this builder.
*/
def putDoubles(array: Array[Double])(implicit byteOrder: ByteOrder): this.type =
putDoubles(array, 0, array.length)(byteOrder)
/**
* Add a number of Doubles from an array to this builder.
*/
def putDoubles(array: Array[Double], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type =
fillByteBuffer(len * 8, byteOrder) { _.asDoubleBuffer.put(array, start, len) }
def clear(): Unit = {
_builder.clear
_length = 0
@ -513,4 +696,13 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
ByteStrings(bytestrings, _length)
}
/**
* Directly wraps this ByteStringBuilder in an OutputStream. Write
* operations on the stream are forwarded to the builder.
*/
def asOutputStream: java.io.OutputStream = new java.io.OutputStream {
def write(b: Int): Unit = builder += b.toByte
override def write(b: Array[Byte], off: Int, len: Int): Unit = { builder.putBytes(b, off, len) }
}
}

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

@ -1,85 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
import scala.util.continuations._
import akka.dispatch.MessageDispatcher
//FIXME Needs docs
package object cps {
def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in)
def loopC[A, U](block: U @cps[A])(implicit loop: CPSLoop[A], dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] =
loop.loopC(block)
def whileC[A, U](test: Boolean)(block: U @cps[A])(implicit loop: CPSLoop[A], dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] =
loop.whileC(test)(block)
def repeatC[A, U](times: Int)(block: U @cps[A])(implicit loop: CPSLoop[A], dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] =
loop.repeatC(times)(block)
}
package cps {
object CPSLoop extends DefaultCPSLoop {
implicit object FutureCPSLoop extends FutureCPSLoop
}
trait CPSLoop[A] {
def loopC[U](block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A]
def whileC[U](test: Boolean)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A]
def repeatC[U](times: Int)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A]
}
import akka.dispatch.{ Future, Promise }
class FutureCPSLoop extends CPSLoop[Future[Any]] {
def loopC[U](block: U @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[Future[Any]] =
shift { c: (Unit Future[Any])
Future(reify(block) flatMap (_ reify(loopC(block))) foreach c)
}
def whileC[U](test: Boolean)(block: U @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[Future[Any]] =
shift { c: (Unit Future[Any])
if (test)
Future(reify(block) flatMap (_ reify(whileC(test)(block))) foreach c)
else
Promise() success (shiftUnitR[Unit, Future[Any]](()) foreach c)
}
def repeatC[U](times: Int)(block: U @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[Future[Any]] =
shift { c: (Unit Future[Any])
if (times > 0)
Future(reify(block) flatMap (_ reify(repeatC(times - 1)(block))) foreach c)
else
Promise() success (shiftUnitR[Unit, Future[Any]](()) foreach c)
}
}
trait DefaultCPSLoop {
implicit def defaultCPSLoop[A] = new CPSLoop[A] {
def loopC[U](block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] = {
block
loopC(block)
}
def whileC[U](test: Boolean)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] = {
if (test) {
block
whileC(test)(block)
}
}
def repeatC[U](times: Int)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] = {
if (times > 0) {
block
repeatC(times - 1)(block)
}
}
}
}
}

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

@ -1,12 +1,11 @@
package akka.agent
import akka.dispatch.Await
import akka.util.Duration
import akka.dispatch.{ Await, Future }
import akka.util.{ Duration, Timeout }
import akka.util.duration._
import akka.util.Timeout
import akka.testkit._
import scala.concurrent.stm._
import java.util.concurrent.CountDownLatch
import java.util.concurrent.{ CountDownLatch, TimeUnit }
class CountDownFunction[A](num: Int = 1) extends Function1[A, A] {
val latch = new CountDownLatch(num)
@ -36,14 +35,15 @@ class AgentSpec extends AkkaSpec {
"maintain order between send and sendOff" in {
val countDown = new CountDownFunction[String]
val l1, l2 = new CountDownLatch(1)
val agent = Agent("a")
agent send (_ + "b")
val longRunning = (s: String) { Thread.sleep(2000); s + "c" }
agent sendOff longRunning
agent.sendOff((s: String) { l1.countDown; l2.await(5, TimeUnit.SECONDS); s + "c" })
l1.await(5, TimeUnit.SECONDS)
agent send (_ + "d")
agent send countDown
l2.countDown
countDown.await(5 seconds)
agent() must be("abcd")
@ -51,16 +51,17 @@ class AgentSpec extends AkkaSpec {
}
"maintain order between alter and alterOff" in {
val l1, l2 = new CountDownLatch(1)
val agent = Agent("a")
val r1 = agent.alter(_ + "b")(5000)
val r2 = agent.alterOff((s: String) { Thread.sleep(2000); s + "c" })(5000)
val r2 = agent.alterOff((s: String) { l1.countDown; l2.await(5, TimeUnit.SECONDS); s + "c" })(5000)
l1.await(5, TimeUnit.SECONDS)
val r3 = agent.alter(_ + "d")(5000)
val result = Future.sequence(Seq(r1, r2, r3)).map(_.mkString(":"))
l2.countDown
Await.result(r1, 5 seconds) must be === "ab"
Await.result(r2, 5 seconds) must be === "abc"
Await.result(r3, 5 seconds) must be === "abcd"
Await.result(result, 5 seconds) must be === "ab:abc:abcd"
agent() must be("abcd")

View file

@ -19,10 +19,10 @@ class ProducerRegistryTest extends WordSpec with MustMatchers with SharedCamelSy
"register a started SendProcessor for the producer, which is stopped when the actor is stopped" in {
val actorRef = newEmptyActor
val processor = registerProcessorFor(actorRef)
camel.awaitActivation(actorRef, 1 second)
camel.awaitActivation(actorRef, 5 second)
processor.isStarted must be(true)
system.stop(actorRef)
camel.awaitDeactivation(actorRef, 1 second)
camel.awaitDeactivation(actorRef, 5 second)
(processor.isStopping || processor.isStopped) must be(true)
}
"remove and stop the SendProcessor if the actorRef is registered" in {

View file

@ -8,12 +8,24 @@
akka {
cluster {
# node to join - the full URI defined by a string on the form of "akka://system@hostname:port"
# leave as empty string if the node should be a singleton cluster
node-to-join = ""
# Initial contact points of the cluster. Nodes to join at startup if auto-join = on.
# The seed nodes also play the role of deputy nodes (the nodes responsible
# for breaking network partitions).
# Comma separated full URIs defined by a string on the form of "akka://system@hostname:port"
# Leave as empty if the node should be a singleton cluster.
seed-nodes = []
# should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN?
auto-down = on
# how long to wait for one of the seed nodes to reply to initial join request
seed-node-timeout = 5s
# Automatic join the seed-nodes at startup.
# If seed-nodes is empty it will join itself and become a single node cluster.
auto-join = on
# Should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN?
# Using auto-down implies that two separate clusters will automatically be formed in case of
# network partition.
auto-down = off
# the number of gossip daemon actors
nr-of-gossip-daemons = 4
@ -36,6 +48,15 @@ akka {
# how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring?
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
# Gossip to random node with newer or older state information, if any with some
# this probability. Otherwise Gossip to any random live node.
# Probability value is between 0.0 and 1.0. 0.0 means never, 1.0 means always.
gossip-different-view-probability = 0.8
failure-detector {
# defines the failure detector threshold
@ -43,9 +64,23 @@ 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
implementation-class = ""
# 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
}

View file

@ -7,50 +7,98 @@ package akka.cluster
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
* 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.
*
* 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
*
* @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)
*
* @param 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(
val system: ActorSystem,
val threshold: Int = 8,
val maxSampleSize: Int = 1000,
val timeMachine: () Long = System.currentTimeMillis) extends FailureDetector {
val threshold: Double,
val maxSampleSize: Int,
val minStdDeviation: Duration,
val acceptableHeartbeatPause: Duration,
val firstHeartbeatEstimate: Duration,
val clock: () Long = AccrualFailureDetector.realClock) extends FailureDetector {
import AccrualFailureDetector._
/**
* Constructor that picks configuration from the settings.
*/
def this(
system: ActorSystem,
settings: ClusterSettings,
timeMachine: () Long = System.currentTimeMillis) =
settings: ClusterSettings) =
this(
system,
settings.FailureDetectorThreshold,
settings.FailureDetectorMaxSampleSize,
timeMachine)
private final val PhiFactor = 1.0 / math.log(10.0)
threshold = settings.FailureDetectorThreshold,
maxSampleSize = settings.FailureDetectorMaxSampleSize,
minStdDeviation = settings.FailureDetectorMinStdDeviation,
acceptableHeartbeatPause = settings.FailureDetectorAcceptableHeartbeatPause,
firstHeartbeatEstimate = settings.HeartbeatInterval,
clock = AccrualFailureDetector.realClock)
private val log = Logging(system, "FailureDetector")
/**
* Holds the failure statistics for a specific node Address.
*/
private case class FailureStats(mean: Double = 0.0, variance: Double = 0.0, deviation: Double = 0.0)
// guess statistics for first heartbeat,
// important so that connections with only one heartbeat becomes unavailble
private val failureStatsFirstHeartbeat = FailureStats(mean = 1000.0)
// 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
@ -58,10 +106,8 @@ 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]],
timestamps: Map[Address, Long] = Map.empty[Address, Long],
explicitRemovals: Set[Address] = Set.empty[Address])
history: Map[Address, HeartbeatHistory] = Map.empty,
timestamps: Map[Address, Long] = Map.empty[Address, Long])
private val state = new AtomicReference[State](State())
@ -78,96 +124,73 @@ class AccrualFailureDetector(
final def heartbeat(connection: Address) {
log.debug("Heartbeat from connection [{}] ", connection)
val timestamp = clock()
val oldState = state.get
val latestTimestamp = oldState.timestamps.get(connection)
if (latestTimestamp.isEmpty) {
// this is heartbeat from a new connection
// add starter records for this new connection
val newState = oldState copy (
version = oldState.version + 1,
failureStats = oldState.failureStats + (connection -> failureStatsFirstHeartbeat),
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 {
// 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 = {
val newMean: Double = newIntervalsForConnection.sum.toDouble / newIntervalsForConnection.size
val oldConnectionFailureStats = oldState.failureStats.get(connection).getOrElse {
throw new IllegalStateException("Can't calculate new failure statistics due to missing heartbeat history")
}
val deviationSum = (0.0d /: newIntervalsForConnection) { (mean, interval)
mean + interval.toDouble - newMean
}
val newVariance: Double = deviationSum / newIntervalsForConnection.size
val newDeviation: Double = math.sqrt(newVariance)
val newFailureStats = oldConnectionFailureStats copy (mean = newMean, deviation = newDeviation, variance = newVariance)
oldState.failureStats + (connection -> newFailureStats)
}
val newState = oldState copy (version = oldState.version + 1,
failureStats = newFailureStats,
intervalHistory = oldState.intervalHistory + (connection -> newIntervalsForConnection),
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
val newHistory = oldState.timestamps.get(connection) match {
case None
// this is heartbeat from a new connection
// add starter records for this new connection
firstHeartbeat
case Some(latestTimestamp)
// this is a known connection
val interval = timestamp - latestTimestamp
oldState.history(connection) :+ interval
}
val newState = oldState copy (version = oldState.version + 1,
history = oldState.history + (connection -> newHistory),
timestamps = oldState.timestamps + (connection -> timestamp)) // record new timestamp
// 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 healthy.
* <p/>
* Implementations of 'Cumulative Distribution Function' for Exponential Distribution.
* For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597].
*/
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.0 // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
else {
val timestampDiff = timeMachine() - oldTimestamp.get
if (oldTimestamp.isEmpty) 0.0 // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
else {
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)
if (mean == 0.0) 0.0
else PhiFactor * timestampDiff / mean
}
val φ = phi(timeDiff, mean + acceptableHeartbeatPauseMillis, stdDeviation)
// FIXME change to debug log level, when failure detector is stable
log.info("Phi value [{}] and threshold [{}] for connection [{}] ", phi, threshold, connection)
phi
// 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 + ")")
φ
}
}
private[cluster] def phi(timeDiff: Long, mean: Double, stdDeviation: Double): Double = {
val cdf = cumulativeDistributionFunction(timeDiff, mean, stdDeviation)
-math.log10(1.0 - cdf)
}
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)))
}
/**
@ -178,15 +201,88 @@ class AccrualFailureDetector(
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,
timestamps = oldState.timestamps - connection,
explicitRemovals = oldState.explicitRemovals + connection)
history = oldState.history - connection,
timestamps = oldState.timestamps - connection)
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) remove(connection) // recur
}
}
def reset(): Unit = {
@tailrec
def doReset(): Unit = {
val oldState = state.get
val newState = oldState.copy(version = oldState.version + 1, history = Map.empty, timestamps = Map.empty)
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) doReset() // recur
}
log.debug("Resetting failure detector")
doReset()
}
}
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,24 +13,30 @@ import akka.actor.AddressFromURIString
class ClusterSettings(val config: Config, val systemName: String) {
import config._
final val FailureDetectorThreshold = getInt("akka.cluster.failure-detector.threshold")
final val FailureDetectorThreshold = getDouble("akka.cluster.failure-detector.threshold")
final val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size")
final val FailureDetectorImplementationClass: Option[String] = getString("akka.cluster.failure-detector.implementation-class") match {
case "" None
case fqcn Some(fqcn)
}
final val NodeToJoin: Option[Address] = getString("akka.cluster.node-to-join") match {
case "" None
case AddressFromURIString(addr) Some(addr)
}
final val PeriodicTasksInitialDelay = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
final val GossipInterval = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
final val HeartbeatInterval = Duration(getMilliseconds("akka.cluster.heartbeat-interval"), MILLISECONDS)
final val LeaderActionsInterval = Duration(getMilliseconds("akka.cluster.leader-actions-interval"), MILLISECONDS)
final val UnreachableNodesReaperInterval = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS)
final val NrOfGossipDaemons = getInt("akka.cluster.nr-of-gossip-daemons")
final val NrOfDeputyNodes = getInt("akka.cluster.nr-of-deputy-nodes")
final val AutoDown = getBoolean("akka.cluster.auto-down")
final val SchedulerTickDuration = Duration(getMilliseconds("akka.cluster.scheduler.tick-duration"), MILLISECONDS)
final val SchedulerTicksPerWheel = getInt("akka.cluster.scheduler.ticks-per-wheel")
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 SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map {
case AddressFromURIString(addr) addr
}.toIndexedSeq
final val SeedNodeTimeout: Duration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS)
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 AutoJoin: Boolean = getBoolean("akka.cluster.auto-join")
final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down")
final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS)
final val GossipDifferentViewProbability: Double = getDouble("akka.cluster.gossip-different-view-probability")
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

@ -25,4 +25,9 @@ trait FailureDetector {
* Removes the heartbeat management for a connection.
*/
def remove(connection: Address): Unit
/**
* Removes all connections and starts over.
*/
def reset(): Unit
}

View file

@ -37,7 +37,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec
"Client of a 4 node cluster" must {
"be able to DOWN a node that is UNREACHABLE (killed)" taggedAs LongRunningTest in {
val thirdAddress = node(third).address
val thirdAddress = address(third)
awaitClusterUp(first, second, third, fourth)
runOn(first) {
@ -47,23 +47,23 @@ abstract class ClientDowningNodeThatIsUnreachableSpec
// 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)
}
runOn(third) {
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
}
runOn(second, fourth) {
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
}
testConductor.enter("await-completion")
enterBarrier("await-completion")
}
}
}

View file

@ -37,13 +37,13 @@ abstract class ClientDowningNodeThatIsUpSpec
"Client of a 4 node cluster" must {
"be able to DOWN a node that is UP (healthy and available)" taggedAs LongRunningTest in {
val thirdAddress = node(third).address
val thirdAddress = address(third)
awaitClusterUp(first, second, third, fourth)
runOn(first) {
// mark 'third' node as DOWN
cluster.down(thirdAddress)
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
markNodeAsUnavailable(thirdAddress)
@ -52,16 +52,16 @@ abstract class ClientDowningNodeThatIsUpSpec
}
runOn(third) {
testConductor.enter("down-third-node")
enterBarrier("down-third-node")
}
runOn(second, fourth) {
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

@ -39,19 +39,19 @@ abstract class ConvergenceSpec
"A cluster of 3 members" must {
"reach initial convergence" taggedAs LongRunningTest ignore {
"reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(fourth) {
// doesn't join immediately
}
testConductor.enter("after-1")
enterBarrier("after-1")
}
"not reach convergence while any nodes are unreachable" taggedAs LongRunningTest ignore {
val thirdAddress = node(third).address
testConductor.enter("before-shutdown")
"not reach convergence while any nodes are unreachable" taggedAs LongRunningTest in {
val thirdAddress = address(third)
enterBarrier("before-shutdown")
runOn(first) {
// kill 'third' node
@ -60,15 +60,13 @@ abstract class ConvergenceSpec
}
runOn(first, second) {
val firstAddress = node(first).address
val secondAddress = node(second).address
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(Seq(firstAddress, secondAddress))
awaitSeenSameState(first, second)
// still one unreachable
cluster.latestGossip.overview.unreachable.size must be(1)
cluster.latestGossip.overview.unreachable.head.address must be(thirdAddress)
@ -78,30 +76,26 @@ abstract class ConvergenceSpec
}
}
testConductor.enter("after-2")
enterBarrier("after-2")
}
"not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest ignore {
"not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest in {
runOn(fourth) {
// try to join
cluster.join(node(first).address)
cluster.join(first)
}
val firstAddress = node(first).address
val secondAddress = node(second).address
val fourthAddress = node(fourth).address
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(Seq(firstAddress, secondAddress, fourthAddress))
memberStatus(firstAddress) must be(Some(MemberStatus.Up))
memberStatus(secondAddress) must be(Some(MemberStatus.Up))
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(fourthAddress) must be(Some(MemberStatus.Joining))
memberStatus(fourth) must be(Some(MemberStatus.Joining))
// still no convergence
cluster.convergence.isDefined must be(false)
}
@ -116,7 +110,7 @@ abstract class ConvergenceSpec
}
}
testConductor.enter("after-3")
enterBarrier("after-3")
}
}
}

View file

@ -55,7 +55,7 @@ trait AccrualFailureDetectorStrategy extends FailureDetectorStrategy { self: Mul
override val failureDetector: FailureDetector = new AccrualFailureDetector(system, new ClusterSettings(system.settings.config, system.name))
override def markNodeAsAvailable(address: Address): Unit = { /* no-op */ }
override def markNodeAsAvailable(address: Address): Unit = ()
override def markNodeAsUnavailable(address: Address): Unit = { /* no-op */ }
override def markNodeAsUnavailable(address: Address): Unit = ()
}

View file

@ -1,65 +0,0 @@
/**
* 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 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 GossipingWithAccrualFailureDetectorMultiJvmNode1 extends GossipingAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
class GossipingWithAccrualFailureDetectorMultiJvmNode2 extends GossipingAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
class GossipingWithAccrualFailureDetectorMultiJvmNode3 extends GossipingAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
abstract class GossipingAccrualFailureDetectorSpec
extends MultiNodeSpec(GossipingAccrualFailureDetectorMultiJvmSpec)
with MultiNodeClusterSpec {
import GossipingAccrualFailureDetectorMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
"A Gossip-driven Failure Detector" must {
"receive gossip 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 gossip
cluster.failureDetector.isAvailable(firstAddress) must be(true)
cluster.failureDetector.isAvailable(secondAddress) must be(true)
cluster.failureDetector.isAvailable(thirdAddress) must be(true)
testConductor.enter("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)
}
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)
}
testConductor.enter("after-2")
}
}
}

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

@ -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._
object JoinSeedNodeMultiJvmSpec extends MultiNodeConfig {
val seed1 = role("seed1")
val seed2 = role("seed2")
val ordinary1 = role("ordinary1")
val ordinary2 = role("ordinary2")
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("akka.cluster.auto-join = on")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class JoinSeedNodeMultiJvmNode1 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy
class JoinSeedNodeMultiJvmNode2 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy
class JoinSeedNodeMultiJvmNode3 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy
class JoinSeedNodeMultiJvmNode4 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy
abstract class JoinSeedNodeSpec
extends MultiNodeSpec(JoinSeedNodeMultiJvmSpec)
with MultiNodeClusterSpec {
import JoinSeedNodeMultiJvmSpec._
override def seedNodes = IndexedSeq(seed1, seed2)
"A cluster with configured seed nodes" must {
"start the seed nodes sequentially" taggedAs LongRunningTest in {
runOn(seed1) {
startClusterNode()
}
enterBarrier("seed1-started")
runOn(seed2) {
startClusterNode()
}
enterBarrier("seed2-started")
runOn(seed1, seed2) {
awaitUpConvergence(2)
}
enterBarrier("after-1")
}
"join the seed nodes at startup" taggedAs LongRunningTest in {
startClusterNode()
enterBarrier("all-started")
awaitUpConvergence(4)
enterBarrier("after-2")
}
}
}

View file

@ -17,7 +17,7 @@ object JoinTwoClustersMultiJvmSpec extends MultiNodeConfig {
val c1 = role("c1")
val c2 = role("c2")
commonConfig(debugConfig(on = true).withFallback(MultiNodeClusterSpec.clusterConfig))
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
@ -33,10 +33,6 @@ abstract class JoinTwoClustersSpec
import JoinTwoClustersMultiJvmSpec._
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 {
@ -44,16 +40,16 @@ abstract class JoinTwoClustersSpec
runOn(a1, b1, c1) {
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
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,20 +71,20 @@ abstract class JoinTwoClustersSpec
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

@ -0,0 +1,293 @@
/**
* 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.ActorSystem
import akka.util.Deadline
import java.util.concurrent.TimeoutException
import scala.collection.immutable.SortedSet
import akka.dispatch.Await
import akka.util.Duration
import java.util.concurrent.TimeUnit
import akka.remote.testconductor.RoleName
object LargeClusterMultiJvmSpec extends MultiNodeConfig {
// each jvm simulates a datacenter with many nodes
val firstDatacenter = role("first-datacenter")
val secondDatacenter = role("second-datacenter")
val thirdDatacenter = role("third-datacenter")
val fourthDatacenter = role("fourth-datacenter")
val fifthDatacenter = role("fifth-datacenter")
// Note that this test uses default configuration,
// not MultiNodeClusterSpec.clusterConfig
commonConfig(ConfigFactory.parseString("""
# Number of ActorSystems in each jvm, can be specified as
# system property when running real tests. Many nodes
# will take long time and consume many threads.
# 10 => 50 nodes is possible to run on one machine.
akka.test.large-cluster-spec.nodes-per-datacenter = 2
akka.cluster {
gossip-interval = 500 ms
auto-join = off
failure-detector.threshold = 4
}
akka.loglevel = INFO
akka.actor.default-dispatcher.fork-join-executor.parallelism-max = 2
akka.scheduler.tick-duration = 33 ms
akka.remote.netty.execution-pool-size = 0
# don't use testconductor transport in this test, especially not
# when using use-dispatcher-for-io
akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
# Using a separate dispatcher for netty io doesn't reduce number
# of needed threads
# akka.remote.netty.use-dispatcher-for-io=akka.test.io-dispatcher
# akka.test.io-dispatcher.fork-join-executor {
# parallelism-min = 100
# parallelism-max = 100
# }
"""))
}
class LargeClusterMultiJvmNode1 extends LargeClusterSpec with AccrualFailureDetectorStrategy
class LargeClusterMultiJvmNode2 extends LargeClusterSpec with AccrualFailureDetectorStrategy
class LargeClusterMultiJvmNode3 extends LargeClusterSpec with AccrualFailureDetectorStrategy
class LargeClusterMultiJvmNode4 extends LargeClusterSpec with AccrualFailureDetectorStrategy
class LargeClusterMultiJvmNode5 extends LargeClusterSpec with AccrualFailureDetectorStrategy
abstract class LargeClusterSpec
extends MultiNodeSpec(LargeClusterMultiJvmSpec)
with MultiNodeClusterSpec {
import LargeClusterMultiJvmSpec._
var systems: IndexedSeq[ActorSystem] = IndexedSeq(system)
val nodesPerDatacenter = system.settings.config.getInt(
"akka.test.large-cluster-spec.nodes-per-datacenter")
/**
* Since we start some ActorSystems/Clusters outside of the
* MultiNodeClusterSpec control we can't use use the mechanism
* defined in MultiNodeClusterSpec to inject failure detector etc.
* Use ordinary Cluster extension with default AccrualFailureDetector.
*/
override def cluster: Cluster = Cluster(system)
override def atTermination(): Unit = {
systems foreach { _.shutdown }
val shutdownTimeout = 20.seconds
val deadline = Deadline.now + shutdownTimeout
systems.foreach { sys
if (sys.isTerminated)
() // already done
else if (deadline.isOverdue)
sys.log.warning("Failed to shutdown [{}] within [{}]", sys.name, shutdownTimeout)
else {
try sys.awaitTermination(deadline.timeLeft) catch {
case _: TimeoutException sys.log.warning("Failed to shutdown [{}] within [{}]", sys.name, shutdownTimeout)
}
}
}
}
def startupSystems(): Unit = {
// one system is already started by the multi-node test
for (n 2 to nodesPerDatacenter)
systems :+= ActorSystem(myself.name + "-" + n, system.settings.config)
// Initialize the Cluster extensions, i.e. startup the clusters
systems foreach { Cluster(_) }
}
def expectedMaxDuration(totalNodes: Int): Duration =
5.seconds + (2.seconds * totalNodes)
def joinAll(from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = {
val joiningClusters = systems.map(Cluster(_)).toSet
join(joiningClusters, from, to, totalNodes, runOnRoles: _*)
}
def join(joiningClusterNodes: Set[Cluster], from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = {
runOnRoles must contain(from)
runOnRoles must contain(to)
runOn(runOnRoles: _*) {
systems.size must be(nodesPerDatacenter) // make sure it is initialized
val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet)
val startGossipCounts = Map.empty[Cluster, Long] ++
clusterNodes.map(c (c -> c.receivedGossipCount))
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c)
val startTime = System.nanoTime
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
val latch = TestLatch(clusterNodes.size)
clusterNodes foreach { c
c.registerListener(new MembershipChangeListener {
override def notify(members: SortedSet[Member]): Unit = {
if (!latch.isOpen && members.size == totalNodes && members.forall(_.status == MemberStatus.Up)) {
log.debug("All [{}] nodes Up in [{}], it took [{}], received [{}] gossip messages",
totalNodes, c.selfAddress, tookMillis, gossipCount(c))
latch.countDown()
}
}
})
}
runOn(from) {
clusterNodes foreach { _ join to }
}
Await.ready(latch, remaining)
awaitCond(clusterNodes.forall(_.convergence.isDefined))
val counts = clusterNodes.map(gossipCount(_))
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / clusterNodes.size, counts.min, counts.max)
log.info("Convergence of [{}] nodes reached, it took [{}], received [{}] gossip messages per node",
totalNodes, tookMillis, formattedStats)
}
}
"A large cluster" must {
"join all nodes in first-datacenter to first-datacenter" taggedAs LongRunningTest in {
runOn(firstDatacenter) {
startupSystems()
startClusterNode()
}
enterBarrier("first-datacenter-started")
val totalNodes = nodesPerDatacenter
within(expectedMaxDuration(totalNodes)) {
joinAll(from = firstDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter)
enterBarrier("first-datacenter-joined")
}
}
"join all nodes in second-datacenter to first-datacenter" taggedAs LongRunningTest in {
runOn(secondDatacenter) {
startupSystems()
}
enterBarrier("second-datacenter-started")
val totalNodes = nodesPerDatacenter * 2
within(expectedMaxDuration(totalNodes)) {
joinAll(from = secondDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter, secondDatacenter)
enterBarrier("second-datacenter-joined")
}
}
"join all nodes in third-datacenter to first-datacenter" taggedAs LongRunningTest in {
runOn(thirdDatacenter) {
startupSystems()
}
enterBarrier("third-datacenter-started")
val totalNodes = nodesPerDatacenter * 3
within(expectedMaxDuration(totalNodes)) {
joinAll(from = thirdDatacenter, to = firstDatacenter, totalNodes,
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter)
enterBarrier("third-datacenter-joined")
}
}
"join all nodes in fourth-datacenter to first-datacenter" taggedAs LongRunningTest in {
runOn(fourthDatacenter) {
startupSystems()
}
enterBarrier("fourth-datacenter-started")
val totalNodes = nodesPerDatacenter * 4
within(expectedMaxDuration(totalNodes)) {
joinAll(from = fourthDatacenter, to = firstDatacenter, totalNodes,
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter)
enterBarrier("fourth-datacenter-joined")
}
}
"join nodes one by one from fifth-datacenter to first-datacenter" taggedAs LongRunningTest in {
runOn(fifthDatacenter) {
startupSystems()
}
enterBarrier("fifth-datacenter-started")
// enough to join a few one-by-one (takes too long time otherwise)
val (bulk, oneByOne) = systems.splitAt(systems.size - 3)
if (bulk.nonEmpty) {
val totalNodes = nodesPerDatacenter * 4 + bulk.size
within(expectedMaxDuration(totalNodes)) {
val joiningClusters = ifNode(fifthDatacenter)(bulk.map(Cluster(_)).toSet)(Set.empty)
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
enterBarrier("fifth-datacenter-joined-" + bulk.size)
}
}
for (i 0 until oneByOne.size) {
val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1
within(expectedMaxDuration(totalNodes)) {
val joiningClusters = ifNode(fifthDatacenter)(Set(Cluster(oneByOne(i))))(Set.empty)
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
enterBarrier("fifth-datacenter-joined-" + (bulk.size + i))
}
}
}
// FIXME sometimes this fails, FD marks nodes from other than second-datacenter as unavailable
"detect failure and auto-down crashed nodes in second-datacenter" taggedAs LongRunningTest ignore {
val unreachableNodes = nodesPerDatacenter
val liveNodes = nodesPerDatacenter * 4
within(20.seconds + expectedMaxDuration(liveNodes)) {
val startGossipCounts = Map.empty[Cluster, Long] ++
systems.map(sys (Cluster(sys) -> Cluster(sys).receivedGossipCount))
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c)
val startTime = System.nanoTime
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
val latch = TestLatch(nodesPerDatacenter)
systems foreach { sys
Cluster(sys).registerListener(new MembershipChangeListener {
override def notify(members: SortedSet[Member]): Unit = {
if (!latch.isOpen && members.size == liveNodes && Cluster(sys).latestGossip.overview.unreachable.size == unreachableNodes) {
log.info("Detected [{}] unreachable nodes in [{}], it took [{}], received [{}] gossip messages",
unreachableNodes, Cluster(sys).selfAddress, tookMillis, gossipCount(Cluster(sys)))
latch.countDown()
}
}
})
}
runOn(firstDatacenter) {
testConductor.shutdown(secondDatacenter, 0)
}
enterBarrier("second-datacenter-shutdown")
runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) {
Await.ready(latch, remaining)
awaitCond(systems.forall(Cluster(_).convergence.isDefined))
val counts = systems.map(sys gossipCount(Cluster(sys)))
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / nodesPerDatacenter, counts.min, counts.max)
log.info("Convergence of [{}] nodes reached after failure, it took [{}], received [{}] gossip messages per node",
liveNodes, tookMillis, formattedStats)
}
enterBarrier("after-6")
}
}
}
}

View file

@ -42,11 +42,11 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec
"be able to DOWN a 'last' node that is UNREACHABLE" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third, fourth)
val fourthAddress = node(fourth).address
val fourthAddress = address(fourth)
runOn(first) {
// kill 'fourth' node
testConductor.shutdown(fourth, 0)
testConductor.enter("down-fourth-node")
enterBarrier("down-fourth-node")
// mark the node as unreachable in the failure detector
markNodeAsUnavailable(fourthAddress)
@ -57,26 +57,26 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec
}
runOn(fourth) {
testConductor.enter("down-fourth-node")
enterBarrier("down-fourth-node")
}
runOn(second, third) {
testConductor.enter("down-fourth-node")
enterBarrier("down-fourth-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
}
testConductor.enter("await-completion-1")
enterBarrier("await-completion-1")
}
"be able to DOWN a 'middle' node that is UNREACHABLE" taggedAs LongRunningTest in {
val secondAddress = node(second).address
val secondAddress = address(second)
testConductor.enter("before-down-second-node")
enterBarrier("before-down-second-node")
runOn(first) {
// kill 'second' node
testConductor.shutdown(second, 0)
testConductor.enter("down-second-node")
enterBarrier("down-second-node")
// mark the node as unreachable in the failure detector
markNodeAsUnavailable(secondAddress)
@ -87,16 +87,16 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec
}
runOn(second) {
testConductor.enter("down-second-node")
enterBarrier("down-second-node")
}
runOn(third) {
testConductor.enter("down-second-node")
enterBarrier("down-second-node")
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds)
}
testConductor.enter("await-completion-2")
enterBarrier("await-completion-2")
}
}
}

View file

@ -50,7 +50,7 @@ abstract class LeaderElectionSpec
assertLeaderIn(sortedRoles)
}
testConductor.enter("after")
enterBarrier("after-1")
}
def shutdownLeaderAndVerifyNewLeader(alreadyShutdown: Int): Unit = {
@ -63,44 +63,46 @@ abstract class LeaderElectionSpec
myself match {
case `controller`
val leaderAddress = node(leader).address
testConductor.enter("before-shutdown")
val leaderAddress = address(leader)
enterBarrier("before-shutdown")
testConductor.shutdown(leader, 0)
testConductor.enter("after-shutdown", "after-down", "completed")
enterBarrier("after-shutdown", "after-down", "completed")
markNodeAsUnavailable(leaderAddress)
case `leader`
testConductor.enter("before-shutdown", "after-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 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 = 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,102 @@
/**
* 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("""
# turn off unreachable reaper
akka.cluster.unreachable-nodes-reaper-interval = 300 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
"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
within(leaderHandoffWaitingTime) {
if (cluster.isLeader) {
enterBarrier("registered-listener")
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)
// verify that the LEADER is REMOVED
awaitCond(cluster.status == MemberStatus.Removed)
} else {
val leavingLatch = TestLatch()
val exitingLatch = TestLatch()
val expectedAddresses = roles.toSet map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
def check(status: MemberStatus): Boolean =
(members.map(_.address) == expectedAddresses &&
members.exists(m m.address == oldLeaderAddress && m.status == status))
if (check(MemberStatus.Leaving)) leavingLatch.countDown()
if (check(MemberStatus.Exiting)) exitingLatch.countDown()
}
})
enterBarrier("registered-listener")
enterBarrier("leader-left")
// verify that the LEADER is LEAVING
leavingLatch.await
// verify that the LEADER is EXITING
exitingLatch.await
// verify that the LEADER is no longer part of the 'members' set
awaitCond(cluster.latestGossip.members.forall(_.address != oldLeaderAddress))
// verify that the LEADER is not part of the 'unreachable' set
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != oldLeaderAddress))
// verify that we have a new LEADER
awaitCond(cluster.leader != oldLeaderAddress)
}
enterBarrier("finished")
}
}
}
}

View file

@ -21,7 +21,7 @@ object MembershipChangeListenerExitingMultiJvmSpec extends MultiNodeConfig {
.withFallback(ConfigFactory.parseString("""
akka.cluster {
leader-actions-interval = 5 s # increase the leader action task interval
unreachable-nodes-reaper-interval = 30 s # turn "off" reaping to unreachable node set
unreachable-nodes-reaper-interval = 300 s # turn "off" reaping to unreachable node set
}
""")
.withFallback(MultiNodeClusterSpec.clusterConfig)))
@ -37,37 +37,33 @@ abstract class MembershipChangeListenerExitingSpec
import MembershipChangeListenerExitingMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
"A registered MembershipChangeListener" must {
"be notified when new node is EXITING" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
testConductor.enter("registered-listener")
cluster.leave(secondAddress)
enterBarrier("registered-listener")
cluster.leave(second)
}
runOn(second) {
testConductor.enter("registered-listener")
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 == secondAddress && m.status == MemberStatus.Exiting))
if (members.size == 3 && members.exists(m m.address == address(second) && m.status == MemberStatus.Exiting))
exitingLatch.countDown()
}
})
testConductor.enter("registered-listener")
enterBarrier("registered-listener")
exitingLatch.await
}
testConductor.enter("finished")
enterBarrier("finished")
}
}
}

View file

@ -30,34 +30,31 @@ abstract class MembershipChangeListenerJoinSpec
import MembershipChangeListenerJoinMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
"A registered MembershipChangeListener" must {
"be notified when new node is JOINING" taggedAs LongRunningTest in {
runOn(first) {
val joinLatch = TestLatch()
val expectedAddresses = Set(firstAddress, secondAddress)
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()
}
})
testConductor.enter("registered-listener")
enterBarrier("registered-listener")
joinLatch.await
}
runOn(second) {
testConductor.enter("registered-listener")
cluster.join(firstAddress)
enterBarrier("registered-listener")
cluster.join(first)
}
awaitUpConvergence(2)
testConductor.enter("after")
enterBarrier("after")
}
}
}

View file

@ -9,6 +9,7 @@ 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")
@ -19,7 +20,7 @@ object MembershipChangeListenerLeavingMultiJvmSpec extends MultiNodeConfig {
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster.leader-actions-interval = 5 s
akka.cluster.unreachable-nodes-reaper-interval = 30 s
akka.cluster.unreachable-nodes-reaper-interval = 300 s # turn "off"
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
@ -34,39 +35,35 @@ abstract class MembershipChangeListenerLeavingSpec
import MembershipChangeListenerLeavingMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
"A registered MembershipChangeListener" must {
"be notified when new node is LEAVING" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
testConductor.enter("registered-listener")
cluster.leave(secondAddress)
enterBarrier("registered-listener")
cluster.leave(second)
}
runOn(second) {
testConductor.enter("registered-listener")
enterBarrier("registered-listener")
}
runOn(third) {
val latch = TestLatch()
val expectedAddresses = Set(firstAddress, secondAddress, thirdAddress)
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 == secondAddress && m.status == MemberStatus.Leaving))
members.exists(m m.address == address(second) && m.status == MemberStatus.Leaving))
latch.countDown()
}
})
testConductor.enter("registered-listener")
enterBarrier("registered-listener")
latch.await
}
testConductor.enter("finished")
enterBarrier("finished")
}
}
}

View file

@ -27,10 +27,6 @@ abstract class MembershipChangeListenerUpSpec
import MembershipChangeListenerUpMultiJvmSpec._
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) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
@ -39,44 +35,44 @@ abstract class MembershipChangeListenerUpSpec
runOn(first, second) {
val latch = TestLatch()
val expectedAddresses = Set(firstAddress, secondAddress)
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()
}
})
testConductor.enter("listener-1-registered")
cluster.join(firstAddress)
enterBarrier("listener-1-registered")
cluster.join(first)
latch.await
}
runOn(third) {
testConductor.enter("listener-1-registered")
enterBarrier("listener-1-registered")
}
testConductor.enter("after-1")
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(firstAddress, secondAddress, thirdAddress)
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()
}
})
testConductor.enter("listener-2-registered")
enterBarrier("listener-2-registered")
runOn(third) {
cluster.join(firstAddress)
cluster.join(first)
}
latch.await
testConductor.enter("after-2")
enterBarrier("after-2")
}
}
}

View file

@ -5,23 +5,28 @@ package akka.cluster
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor.{Address, ExtendedActorSystem}
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-join = off
auto-down = off
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
@ -29,14 +34,65 @@ object MultiNodeClusterSpec {
""")
}
trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
trait MultiNodeClusterSpec extends FailureDetectorStrategy with Suite { self: MultiNodeSpec
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
}
/**
* Make it possible to override/configure seedNodes from tests without
* specifying in config. Addresses are unknown before startup time.
*/
protected def seedNodes: IndexedSeq[RoleName] = IndexedSeq.empty
/**
* The cluster node instance. Needs to be lazily created.
*/
private lazy val clusterNode = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector)
private lazy val clusterNode = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) {
override def seedNodes: IndexedSeq[Address] = {
val testSeedNodes = MultiNodeClusterSpec.this.seedNodes
if (testSeedNodes.isEmpty) super.seedNodes
else testSeedNodes map address
}
}
/**
* Get the cluster node to use.
@ -44,10 +100,15 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
def cluster: Cluster = clusterNode
/**
* Use this method instead of 'cluster.self'
* for the initial startup of the cluster node.
* Use this method for the initial startup of the cluster node.
*/
def startClusterNode(): Unit = cluster.self
def startClusterNode(): Unit = {
if (cluster.latestGossip.members.isEmpty) {
cluster join myself
awaitCond(cluster.latestGossip.members.exists(_.address == address(myself)))
} else
cluster.self
}
/**
* Initialize the cluster with the specified member
@ -71,14 +132,14 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
// make sure that the node-to-join is started before other join
startClusterNode()
}
testConductor.enter(roles.head.name + "-started")
enterBarrier(roles.head.name + "-started")
if (roles.tail.contains(myself)) {
cluster.join(node(roles.head).address)
cluster.join(roles.head)
}
if (upConvergence && roles.contains(myself)) {
awaitUpConvergence(numberOfMembers = roles.length)
}
testConductor.enter(roles.map(_.name).mkString("-") + "-joined")
enterBarrier(roles.map(_.name).mkString("-") + "-joined")
}
/**
@ -129,7 +190,7 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
/**
* Wait until the specified nodes have seen the same gossip overview.
*/
def awaitSeenSameState(addresses: Seq[Address]): Unit = {
def awaitSeenSameState(addresses: Address*): Unit = {
awaitCond {
val seen = cluster.latestGossip.overview.seen
val seenVectorClocks = addresses.flatMap(seen.get(_))
@ -137,7 +198,7 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
}
}
def roleOfLeader(nodesInCluster: Seq[RoleName]): RoleName = {
def roleOfLeader(nodesInCluster: Seq[RoleName] = roles): RoleName = {
nodesInCluster.length must not be (0)
nodesInCluster.sorted.head
}
@ -147,10 +208,9 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { 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

@ -17,7 +17,7 @@ object NodeJoinMultiJvmSpec extends MultiNodeConfig {
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval
.withFallback(MultiNodeClusterSpec.clusterConfig)))
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
class NodeJoinMultiJvmNode1 extends NodeJoinSpec with FailureDetectorPuppetStrategy
@ -29,9 +29,6 @@ abstract class NodeJoinSpec
import NodeJoinMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
"A cluster node" must {
"join another cluster and get status JOINING - when sending a 'Join' command" taggedAs LongRunningTest in {
@ -39,13 +36,15 @@ abstract class NodeJoinSpec
startClusterNode()
}
enterBarrier("first-started")
runOn(second) {
cluster.join(firstAddress)
cluster.join(first)
}
awaitCond(cluster.latestGossip.members.exists { member member.address == secondAddress && member.status == MemberStatus.Joining })
awaitCond(cluster.latestGossip.members.exists { member member.address == address(second) && member.status == MemberStatus.Joining })
testConductor.enter("after")
enterBarrier("after")
}
}
}

View file

@ -18,9 +18,9 @@ object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
abstract class NodeLeavingAndExitingAndBeingRemovedSpec
extends MultiNodeSpec(NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec)
@ -28,38 +28,34 @@ abstract class NodeLeavingAndExitingAndBeingRemovedSpec
import NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
val reaperWaitingTime = 30.seconds.dilated
"A node that is LEAVING a non-singleton cluster" must {
// FIXME make it work and remove ignore
"be moved to EXITING and then to REMOVED by the reaper" taggedAs LongRunningTest ignore {
"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(secondAddress)
cluster.leave(second)
}
testConductor.enter("second-left")
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 != secondAddress), reaperWaitingTime)
awaitCond(cluster.latestGossip.members.forall(_.address != address(second)), reaperWaitingTime)
// verify that the 'second' node is part of the 'unreachable' set
awaitCond(cluster.latestGossip.overview.unreachable.exists(_.status == MemberStatus.Removed), reaperWaitingTime)
// verify node that got removed is 'second' node
val isRemoved = cluster.latestGossip.overview.unreachable.find(_.status == MemberStatus.Removed)
isRemoved must be('defined)
isRemoved.get.address must be(secondAddress)
// verify that the 'second' node is not part of the 'unreachable' set
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != address(second)), reaperWaitingTime)
}
testConductor.enter("finished")
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

@ -18,11 +18,8 @@ object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig {
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
}
""")
# turn off unreachable reaper
akka.cluster.unreachable-nodes-reaper-interval = 300 s""")
.withFallback(MultiNodeClusterSpec.clusterConfig)))
}
@ -36,40 +33,48 @@ abstract class NodeLeavingAndExitingSpec
import NodeLeavingAndExitingMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
"A node that is LEAVING a non-singleton cluster" must {
// FIXME make it work and remove ignore
"be moved to EXITING by the leader" taggedAs LongRunningTest ignore {
"be moved to EXITING by the leader" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
cluster.leave(secondAddress)
}
testConductor.enter("second-left")
runOn(first, third) {
val secondAddess = address(second)
val leavingLatch = TestLatch()
val exitingLatch = TestLatch()
val expectedAddresses = roles.toSet map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
def check(status: MemberStatus): Boolean =
(members.map(_.address) == expectedAddresses &&
members.exists(m m.address == secondAddess && m.status == status))
if (check(MemberStatus.Leaving)) leavingLatch.countDown()
if (check(MemberStatus.Exiting)) exitingLatch.countDown()
}
})
enterBarrier("registered-listener")
// 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(secondAddress)
runOn(third) {
cluster.leave(second)
}
enterBarrier("second-left")
// Verify that 'second' node is set to LEAVING
leavingLatch.await
// Verify that 'second' node is set to EXITING
exitingLatch.await
// 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(secondAddress)
}
testConductor.enter("finished")
// node that is leaving
runOn(second) {
enterBarrier("registered-listener")
enterBarrier("second-left")
}
enterBarrier("finished")
}
}
}

View file

@ -30,31 +30,26 @@ abstract class NodeLeavingSpec
import NodeLeavingMultiJvmSpec._
lazy val firstAddress = node(first).address
lazy val secondAddress = node(second).address
lazy val thirdAddress = node(third).address
"A node that is LEAVING a non-singleton cluster" must {
// FIXME make it work and remove ignore
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest ignore {
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third)
runOn(first) {
cluster.leave(secondAddress)
cluster.leave(second)
}
testConductor.enter("second-left")
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(secondAddress)
hasLeft.get.address must be(address(second))
}
testConductor.enter("finished")
enterBarrier("finished")
}
}
}

View file

@ -26,10 +26,6 @@ abstract class NodeMembershipSpec
import NodeMembershipMultiJvmSpec._
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 {
@ -38,35 +34,35 @@ abstract class NodeMembershipSpec
runOn(first) {
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)
}
testConductor.enter("after-1")
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)
testConductor.enter("after-2")
enterBarrier("after-2")
}
}
}

View file

@ -33,7 +33,7 @@ abstract class NodeUpSpec
awaitClusterUp(first, second)
testConductor.enter("after-1")
enterBarrier("after-1")
}
"be unaffected when joining again" taggedAs LongRunningTest in {
@ -45,12 +45,12 @@ abstract class NodeUpSpec
unexpected.set(members)
}
})
testConductor.enter("listener-registered")
enterBarrier("listener-registered")
runOn(second) {
cluster.join(node(first).address)
cluster.join(first)
}
testConductor.enter("joined-again")
enterBarrier("joined-again")
// let it run for a while to make sure that nothing bad happens
for (n 1 to 20) {
@ -59,7 +59,7 @@ abstract class NodeUpSpec
cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true)
}
testConductor.enter("after-2")
enterBarrier("after-2")
}
}
}

View file

@ -16,6 +16,7 @@ object SingletonClusterMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("""
akka.cluster {
auto-join = on
auto-down = on
failure-detector.threshold = 4
}
@ -38,17 +39,25 @@ abstract class SingletonClusterSpec
"A cluster of 2 nodes" must {
"not be singleton cluster when joined" taggedAs LongRunningTest in {
"become singleton cluster when started with 'auto-join=on' and 'seed-nodes=[]'" taggedAs LongRunningTest in {
startClusterNode()
awaitUpConvergence(1)
cluster.isSingletonCluster must be(true)
enterBarrier("after-1")
}
"not be singleton cluster when joined with other node" taggedAs LongRunningTest in {
awaitClusterUp(first, second)
cluster.isSingletonCluster must be(false)
assertLeader(first, second)
testConductor.enter("after-1")
enterBarrier("after-2")
}
"become singleton cluster when one node is shutdown" taggedAs LongRunningTest in {
runOn(first) {
val secondAddress = node(second).address
val secondAddress = address(second)
testConductor.shutdown(second, 0)
markNodeAsUnavailable(secondAddress)
@ -58,7 +67,7 @@ abstract class SingletonClusterSpec
assertLeader(first)
}
testConductor.enter("after-2")
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,111 @@
/**
* 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
import akka.remote.testconductor.Direction
object SplitBrainMultiJvmSpec 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 {
auto-down = on
failure-detector.threshold = 4
}""")).
withFallback(MultiNodeClusterSpec.clusterConfig))
}
class SplitBrainWithFailureDetectorPuppetMultiJvmNode1 extends SplitBrainSpec with FailureDetectorPuppetStrategy
class SplitBrainWithFailureDetectorPuppetMultiJvmNode2 extends SplitBrainSpec with FailureDetectorPuppetStrategy
class SplitBrainWithFailureDetectorPuppetMultiJvmNode3 extends SplitBrainSpec with FailureDetectorPuppetStrategy
class SplitBrainWithFailureDetectorPuppetMultiJvmNode4 extends SplitBrainSpec with FailureDetectorPuppetStrategy
class SplitBrainWithFailureDetectorPuppetMultiJvmNode5 extends SplitBrainSpec with FailureDetectorPuppetStrategy
class SplitBrainWithAccrualFailureDetectorMultiJvmNode1 extends SplitBrainSpec with AccrualFailureDetectorStrategy
class SplitBrainWithAccrualFailureDetectorMultiJvmNode2 extends SplitBrainSpec with AccrualFailureDetectorStrategy
class SplitBrainWithAccrualFailureDetectorMultiJvmNode3 extends SplitBrainSpec with AccrualFailureDetectorStrategy
class SplitBrainWithAccrualFailureDetectorMultiJvmNode4 extends SplitBrainSpec with AccrualFailureDetectorStrategy
class SplitBrainWithAccrualFailureDetectorMultiJvmNode5 extends SplitBrainSpec with AccrualFailureDetectorStrategy
abstract class SplitBrainSpec
extends MultiNodeSpec(SplitBrainMultiJvmSpec)
with MultiNodeClusterSpec {
import SplitBrainMultiJvmSpec._
val side1 = IndexedSeq(first, second)
val side2 = IndexedSeq(third, fourth, fifth)
"A cluster of 5 members" must {
"reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third, fourth, fifth)
enterBarrier("after-1")
}
"detect network partition and mark nodes on other side as unreachable" taggedAs LongRunningTest in {
val thirdAddress = address(third)
enterBarrier("before-split")
runOn(first) {
// split the cluster in two parts (first, second) / (third, fourth, fifth)
for (role1 side1; role2 side2) {
testConductor.blackhole(role1, role2, Direction.Both).await
}
}
enterBarrier("after-split")
runOn(side1.last) {
for (role side2) markNodeAsUnavailable(role)
}
runOn(side2.last) {
for (role side1) markNodeAsUnavailable(role)
}
runOn(side1: _*) {
awaitCond(cluster.latestGossip.overview.unreachable.map(_.address) == (side2.toSet map address), 20 seconds)
}
runOn(side2: _*) {
awaitCond(cluster.latestGossip.overview.unreachable.map(_.address) == (side1.toSet map address), 20 seconds)
}
enterBarrier("after-2")
}
"auto-down the other nodes and form new cluster with potentially new leader" taggedAs LongRunningTest in {
runOn(side1: _*) {
// auto-down = on
awaitCond(cluster.latestGossip.overview.unreachable.forall(m m.status == MemberStatus.Down), 15 seconds)
cluster.latestGossip.overview.unreachable.map(_.address) must be(side2.toSet map address)
awaitUpConvergence(side1.size, side2 map address)
assertLeader(side1: _*)
}
runOn(side2: _*) {
// auto-down = on
awaitCond(cluster.latestGossip.overview.unreachable.forall(m m.status == MemberStatus.Down), 15 seconds)
cluster.latestGossip.overview.unreachable.map(_.address) must be(side1.toSet map address)
awaitUpConvergence(side2.size, side1 map address)
assertLeader(side2: _*)
}
enterBarrier("after-3")
}
}
}

View file

@ -19,20 +19,21 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
val fourth = role("fourth")
val fifth = role("fifth")
// Note that this test uses default configuration,
// not MultiNodeClusterSpec.clusterConfig
commonConfig(ConfigFactory.parseString("""
akka.cluster {
gossip-interval = 400 ms
nr-of-deputy-nodes = 0
auto-join = off
}
akka.loglevel = INFO
"""))
}
class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
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)
@ -62,7 +63,7 @@ abstract class SunnyWeatherSpec
})
for (n 1 to 30) {
testConductor.enter("period-" + n)
enterBarrier("period-" + n)
unexpected.get must be(null)
awaitUpConvergence(roles.size)
assertLeaderIn(roles)
@ -70,7 +71,7 @@ abstract class SunnyWeatherSpec
1.seconds.sleep
}
testConductor.enter("after")
enterBarrier("after")
}
}
}

View file

@ -0,0 +1,261 @@
/**
* 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")
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
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
// gossip chat will synchronize the views
awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty)
enterBarrier("after-gossip-" + gossipBarrierCounter)
}
runOn(fromRole) {
enterBarrier("before-gossip-" + gossipBarrierCounter)
cluster.gossipTo(toRole) // send gossip
// gossip chat will synchronize the views
awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty)
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 {
runOn(first) {
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, second) {
// gossip chat from the join will synchronize the views
awaitMembers(first, second)
memberStatus(first) must be(Up)
memberStatus(second) must be(Joining)
awaitCond(seenLatestGossip == 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(first, second) {
// gossip chat will synchronize the views
awaitCond(memberStatus(second) == Up)
seenLatestGossip must be(Set(first, second))
memberStatus(first) must be(Up)
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, third) {
// gossip chat from the join will synchronize the views
awaitMembers(first, second, third)
memberStatus(third) must be(Joining)
awaitCond(seenLatestGossip == Set(second, third))
cluster.convergence.isDefined must be(false)
}
enterBarrier("third-joined-second")
second gossipTo first
runOn(first, second) {
// gossip chat will synchronize the views
awaitMembers(first, second, third)
memberStatus(third) must be(Joining)
awaitCond(memberStatus(second) == Up)
seenLatestGossip must be(Set(first, second, third))
cluster.convergence.isDefined must be(true)
}
first gossipTo third
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)
}
// 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")
}
"perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in {
runOn(third) {
markNodeAsUnavailable(second)
cluster.reapUnreachableMembers()
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
seenLatestGossip must be(Set(third))
}
enterBarrier("after-second-unavailble")
third gossipTo first
runOn(first, third) {
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
cluster.convergence.isDefined must be(false)
}
runOn(first) {
cluster.down(second)
awaitMemberStatus(second, Down)
}
enterBarrier("after-second-down")
first gossipTo third
runOn(first, third) {
cluster.latestGossip.overview.unreachable must contain(Member(second, Down))
memberStatus(second) must be(Down)
seenLatestGossip must be(Set(first, third))
cluster.convergence.isDefined must be(true)
}
enterBarrier("after-6")
}
}
}

View file

@ -0,0 +1,137 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.actor.Address
import akka.remote.testconductor.{RoleName, Direction}
import akka.util.duration._
object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
abstract class UnreachableNodeRejoinsClusterSpec
extends MultiNodeSpec(UnreachableNodeRejoinsClusterMultiJvmSpec)
with MultiNodeClusterSpec {
import UnreachableNodeRejoinsClusterMultiJvmSpec._
def allBut(role: RoleName, roles: Seq[RoleName] = roles): Seq[RoleName] = {
roles.filterNot(_ == role)
}
lazy val sortedRoles = roles.sorted
lazy val master = sortedRoles(0)
lazy val victim = sortedRoles(1)
var endBarrierNumber = 0
def endBarrier: Unit = {
endBarrierNumber += 1
enterBarrier("after_" + endBarrierNumber)
}
"A cluster of " + roles.size + " members" must {
"reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(roles:_*)
endBarrier
}
"mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in {
runOn(first) {
// pull network for victim node from all nodes
allBut(victim).foreach { roleName =>
testConductor.blackhole(victim, roleName, Direction.Both).await
}
}
enterBarrier("unplug_victim")
val allButVictim = allBut(victim, sortedRoles)
runOn(victim) {
allButVictim.foreach(markNodeAsUnavailable(_))
within(30 seconds) {
// victim becomes all alone
awaitCond({ val gossip = cluster.latestGossip
gossip.overview.unreachable.size == (roles.size - 1) &&
gossip.members.size == 1 &&
gossip.members.forall(_.status == MemberStatus.Up) })
cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet)
cluster.convergence.isDefined must be(false)
}
}
runOn(allButVictim:_*) {
markNodeAsUnavailable(victim)
within(30 seconds) {
// victim becomes unreachable
awaitCond({ val gossip = cluster.latestGossip
gossip.overview.unreachable.size == 1 &&
gossip.members.size == (roles.size - 1) &&
gossip.members.forall(_.status == MemberStatus.Up) })
awaitSeenSameState(allButVictim map address:_*)
// still one unreachable
cluster.latestGossip.overview.unreachable.size must be(1)
cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address)
// and therefore no convergence
cluster.convergence.isDefined must be(false)
}
}
endBarrier
}
"mark the node as DOWN" taggedAs LongRunningTest in {
runOn(master) {
cluster down victim
}
runOn(allBut(victim):_*) {
awaitUpConvergence(roles.size - 1, Seq(victim))
}
endBarrier
}
"allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in {
runOn(first) {
// put the network back in
allBut(victim).foreach { roleName =>
testConductor.passThrough(victim, roleName, Direction.Both).await
}
}
enterBarrier("plug_in_victim")
runOn(victim) {
cluster join master
}
awaitUpConvergence(roles.size)
endBarrier
}
}
}

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("""
@ -27,33 +30,72 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
timeGenerator
}
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 = new AccrualFailureDetector(system)
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 {
// 1 second ticks
val timeInterval = Vector.fill(30)(1000L)
val fd = new AccrualFailureDetector(system,
timeMachine = fakeTimeGenerator(timeInterval))
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.0)
// let time go
for (n 2 to 8)
fd.phi(conn) must be < (4.0)
for (n 9 to 18)
fd.phi(conn) must be < (8.0)
fd.phi(conn) must be > (8.0)
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 = new AccrualFailureDetector(system,
timeMachine = fakeTimeGenerator(timeInterval))
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.phi(conn) must be > (0.0)
@ -63,8 +105,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
"mark node as available after a series of successful heartbeats" in {
val timeInterval = List[Long](0, 1000, 100, 100)
val fd = new AccrualFailureDetector(system,
timeMachine = fakeTimeGenerator(timeInterval))
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.heartbeat(conn)
@ -73,10 +114,9 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
fd.isAvailable(conn) must be(true)
}
"mark node as dead after explicit removal of connection" in {
"mark node as available after explicit removal of connection" in {
val timeInterval = List[Long](0, 1000, 100, 100, 100)
val fd = new AccrualFailureDetector(system,
timeMachine = fakeTimeGenerator(timeInterval))
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn)
fd.heartbeat(conn)
@ -84,13 +124,12 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
fd.isAvailable(conn) must be(true)
fd.remove(conn)
fd.isAvailable(conn) must be(false)
fd.isAvailable(conn) must be(true)
}
"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 fd = new AccrualFailureDetector(system,
timeMachine = fakeTimeGenerator(timeInterval))
val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval))
fd.heartbeat(conn) //0
@ -101,7 +140,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
fd.remove(conn)
fd.isAvailable(conn) must be(false) //3300
fd.isAvailable(conn) must be(true) //3300
// it receives heartbeat from an explicitly removed node
fd.heartbeat(conn) //4400
@ -112,40 +151,65 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
}
"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, threshold = 3,
timeMachine = fakeTimeGenerator(timeInterval))
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 fd = new AccrualFailureDetector(system, threshold = 3,
timeMachine = fakeTimeGenerator(timeInterval))
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) //6200
fd.heartbeat(conn) //6300
fd.heartbeat(conn) //7300
fd.heartbeat(conn) //7400
fd.isAvailable(conn) must be(false) //8200
fd.heartbeat(conn) //8300
fd.heartbeat(conn) //9300
fd.heartbeat(conn) //9400
fd.isAvailable(conn) must be(true) //7500
fd.isAvailable(conn) must be(true) //9500
}
"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)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
fd.heartbeat(conn)
fd.isAvailable(conn) must be(true)
}
"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.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 fd = new AccrualFailureDetector(system, maxSampleSize = 3,
timeMachine = fakeTimeGenerator(timeInterval))
val fd = createFailureDetector(maxSampleSize = 3, clock = fakeTimeGenerator(timeInterval))
// 100 ms interval
fd.heartbeat(conn) //0
@ -163,4 +227,33 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
}
}
"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,18 +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(None)
NodeToJoin must be(None)
FailureDetectorImplementationClass must be(classOf[AccrualFailureDetector].getName)
FailureDetectorMinStdDeviation must be(100 millis)
FailureDetectorAcceptableHeartbeatPause must be(3 seconds)
SeedNodes must be(Seq.empty[String])
SeedNodeTimeout must be(5 seconds)
PeriodicTasksInitialDelay must be(1 seconds)
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)
AutoJoin must be(true)
AutoDown must be(false)
GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001)
SchedulerTickDuration must be(33 millis)
SchedulerTicksPerWheel must be(512)
}

View file

@ -11,12 +11,13 @@ import akka.actor.ExtendedActorSystem
import akka.actor.Address
import java.util.concurrent.atomic.AtomicInteger
import org.scalatest.BeforeAndAfter
import akka.remote.RemoteActorRefProvider
object ClusterSpec {
val config = """
akka.cluster {
auto-join = off
auto-down = off
nr-of-deputy-nodes = 3
periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks
}
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
@ -31,9 +32,23 @@ object ClusterSpec {
class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
import ClusterSpec._
val selfAddress = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.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))
val deterministicRandom = new AtomicInteger
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], new FailureDetectorPuppet(system)) {
val failureDetector = new FailureDetectorPuppet(system)
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) {
// 3 deputy nodes (addresses index 1, 2, 3)
override def seedNodes = addresses.slice(1, 4)
override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = {
if (addresses.isEmpty) None
@ -48,14 +63,6 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
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
@ -64,41 +71,28 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
else _gossipToDeputyProbablity
}
@volatile
var _unavailable: Set[Address] = Set.empty
override val failureDetector = new FailureDetectorPuppet(system) {
override def isAvailable(connection: Address): Boolean = {
if (_unavailable.contains(connection)) false
else super.isAvailable(connection)
}
}
}
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
cluster._unavailable = Set.empty
addresses foreach failureDetector.remove
deterministicRandom.set(0)
}
"A Cluster" must {
"initially be singleton cluster and reach convergence immediately" in {
cluster.isSingletonCluster must be(true)
"use the address of the remote transport" in {
cluster.selfAddress must be(selfAddress)
}
"initially become singleton cluster when joining itself and reach convergence" in {
cluster.isSingletonCluster must be(false) // auto-join = off
cluster.join(selfAddress)
awaitCond(cluster.isSingletonCluster)
cluster.self.address must be(selfAddress)
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress))
memberStatus(selfAddress) must be(Some(MemberStatus.Joining))
cluster.convergence.isDefined must be(true)
@ -111,12 +105,14 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
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)
expectMsg(GossipTo(addresses(1)))
}
"accept a few more joining nodes" in {
for (a addresses.drop(2)) {
cluster.joining(a)
memberStatus(a) must be(Some(MemberStatus.Joining))
expectMsg(GossipTo(a))
}
cluster.latestGossip.members.map(_.address) must be(addresses.toSet)
}
@ -127,7 +123,6 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
}
"gossip to random live node" in {
cluster.latestGossip.members
cluster.gossip()
cluster.gossip()
cluster.gossip()
@ -141,17 +136,6 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
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))
@ -169,7 +153,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
"gossip to duputy node" in {
cluster._gossipToDeputyProbablity = 1.0 // always
// we have configured 2 deputy nodes
// we have configured 3 deputy nodes (seedNodes)
cluster.gossip() // 1 is deputy
cluster.gossip() // 2 is deputy
cluster.gossip() // 3 is deputy
@ -186,27 +170,11 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
}
"gossip to random unreachable node" in {
val dead = Set(addresses(1))
cluster._unavailable = dead
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))
cluster._unavailable = dead
dead foreach failureDetector.markNodeAsUnavailable
cluster.reapUnreachableMembers()
cluster.latestGossip.overview.unreachable.map(_.address) must be(dead)

View file

@ -57,4 +57,9 @@ class FailureDetectorPuppet(system: ActorSystem, settings: ClusterSettings) exte
log.debug("Removing cluster node [{}]", connection)
connections.remove(connection)
}
def reset(): Unit = {
log.debug("Resetting failure detector")
connections.clear()
}
}

View file

@ -33,12 +33,12 @@ class GossipSpec extends WordSpec with MustMatchers {
val g2 = Gossip(members = SortedSet(a2, c2, e2))
val merged1 = g1 merge g2
merged1.members must be(SortedSet(a1, c1, e2))
merged1.members.toSeq.map(_.status) must be(Seq(Up, Leaving, Up))
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(a1, c1, e2))
merged2.members.toSeq.map(_.status) must be(Seq(Up, Leaving, Up))
merged2.members must be(SortedSet(a2, c1, e1))
merged2.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining))
}
@ -48,12 +48,12 @@ class GossipSpec extends WordSpec with MustMatchers {
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(a1, b2, c1, d2))
merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Up, Removed, Leaving, Removed))
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(a1, b2, c1, d2))
merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Up, Removed, Leaving, Removed))
merged2.overview.unreachable must be(Set(a2, b2, c1, d2))
merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed))
}
@ -62,14 +62,14 @@ class GossipSpec extends WordSpec with MustMatchers {
val g2 = Gossip(members = SortedSet(a2, c2), overview = GossipOverview(unreachable = Set(b2, d2)))
val merged1 = g1 merge g2
merged1.members must be(SortedSet(a1))
merged1.members.toSeq.map(_.status) must be(Seq(Up))
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(a1))
merged2.members.toSeq.map(_.status) must be(Seq(Up))
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))

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,45 +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
import scala.collection.immutable.SortedSet
@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)
(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)
}
}
}

View file

@ -0,0 +1,25 @@
package docs.osgi
case object SomeMessage
class SomeActor extends akka.actor.Actor {
def receive = { case SomeMessage }
}
//#Activator
import akka.actor.{ Props, ActorSystem }
import org.osgi.framework.BundleContext
import akka.osgi.ActorSystemActivator
class Activator extends ActorSystemActivator {
def configure(context: BundleContext, system: ActorSystem) {
// optionally register the ActorSystem in the OSGi Service Registry
registerService(context, system)
val someActor = system.actorOf(Props[SomeActor], name = "someName")
someActor ! SomeMessage
}
}
//#Activator

View file

@ -0,0 +1,14 @@
<?xml version="1.0" encoding="UTF-8"?>
<blueprint xmlns="http://www.osgi.org/xmlns/blueprint/v1.0.0"
xmlns:akka="http://akka.io/xmlns/blueprint/v1.0.0">
<akka:actor-system name="BlueprintSystem" />
<akka:actor-system name="BlueprintSystemWithConfig">
<akka:config>
some.config {
key=value
}
</akka:config>
</akka:actor-system>
</blueprint>

View file

@ -8,3 +8,20 @@ To use Akka in an OSGi environment, the ``org.osgi.framework.bootdelegation``
property must be set to always delegate the ``sun.misc`` package to the boot classloader
instead of resolving it through the normal OSGi class space.
Activator
---------
To bootstrap Akka inside an OSGi environment, you can use the akka.osgi.AkkaSystemActivator class
to conveniently set up the ActorSystem.
.. includecode:: code/osgi/Activator.scala#Activator
Blueprint
---------
For the Apache Aries Blueprint implementation, there's also a namespace handler available. The namespace URI
is http://akka.io/xmlns/blueprint/v1.0.0 and it can be used to set up an ActorSystem.
.. includecode:: code/osgi/blueprint.xml

View file

@ -5,8 +5,7 @@
Cluster Specification
######################
.. note:: *This document describes the new clustering coming in Akka Coltrane and
is not available in the latest stable release)*
.. note:: *This document describes the new clustering coming in Akka Coltrane and is not available in the latest stable release)*
Intro
=====
@ -139,7 +138,7 @@ implementation of `The Phi Accrual Failure Detector`_ by Hayashibara et al.
An accrual failure detector decouple monitoring and interpretation. That makes
them applicable to a wider area of scenarios and more adequate to build generic
failure detection services. The idea is that it is keeping a history of failure
statistics, calculated from heartbeats received from the gossip protocol, and is
statistics, calculated from heartbeats received from other nodes, and is
trying to do educated guesses by taking multiple factors, and how they
accumulate over time, into account in order to come up with a better guess if a
specific node is up or down. Rather than just answering "yes" or "no" to the
@ -164,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
@ -184,14 +183,20 @@ according to the Failure Detector is considered unreachable. This means setting
the unreachable node status to ``down`` automatically.
Seed Nodes
^^^^^^^^^^
The seed nodes are configured contact points for inital join of the cluster.
When a new node is started started it sends a message to all seed nodes and
then sends join command to the one that answers first.
It is possible to turn off automatic join.
Deputy Nodes
^^^^^^^^^^^^
After gossip convergence a set of ``deputy`` nodes for the cluster can be
determined. As with the ``leader``, there is no ``deputy`` election process,
the deputies can always be recognised deterministically by any node whenever there
is gossip convergence. The list of ``deputy`` nodes is simply the N - 1 number
of nodes (e.g. starting with the first node after the ``leader``) in sorted order.
The deputy nodes are the live members of the configured seed nodes.
It is preferred to use deputy nodes in different racks/data centers.
The nodes defined as ``deputy`` nodes are just regular member nodes whose only
"special role" is to help breaking logical partitions as seen in the gossip
@ -214,7 +219,7 @@ nodes involved in a gossip exchange.
Periodically, the default is every 1 second, each node chooses another random
node to initiate a round of gossip with. The choice of node is random but can
also include extra gossiping for unreachable nodes, ``deputy`` nodes, and nodes with
also include extra gossiping for ``deputy`` nodes, and nodes with
either newer or older state versions.
The gossip overview contains the current state version for all nodes and also a
@ -227,18 +232,14 @@ breaking logical partitions as seen in the gossip algorithm defined below.
During each round of gossip exchange the following process is used:
1. Gossip to random live node (if any)
1. Gossip to random node with newer or older state information, if any, based on the
current gossip overview, with some probability. Otherwise Gossip to any random
live node.
2. Gossip to random unreachable node with certain probability depending on the
number of unreachable and live nodes
3. If the node gossiped to at (1) was not a ``deputy`` node, or the number of live
2. If the node gossiped to at (1) was not a ``deputy`` node, or the number of live
nodes is less than number of ``deputy`` nodes, gossip to random ``deputy`` node with
certain probability depending on number of unreachable, ``deputy``, and live nodes.
4. Gossip to random node with newer or older state information, based on the
current gossip overview, with some probability (?)
The gossiper only sends the gossip overview to the chosen node. The recipient of
the gossip can use the gossip overview to determine whether:
@ -302,10 +303,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
@ -314,11 +311,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
@ -339,12 +337,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
^^^^^^^^^^^^
@ -359,9 +357,6 @@ User Actions
- **down**
mark a node as temporarily down
- **remove**
remove a node from the cluster immediately
Leader Actions
^^^^^^^^^^^^^^

View file

@ -48,14 +48,14 @@ At-most-once
Actual transports may provide stronger semantics,
but at-most-once is the semantics you should expect.
The alternatives would be once-and-only-once, which is extremely costly,
The alternatives would be once-and-only-once, which is extremely costly,
or at-least-once which essentially requires idempotency of message processing,
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:
@ -66,4 +66,4 @@ This means that:
5) ``A2`` can see messages from ``A1`` interleaved with messages from ``A3``
6) Since there is no guaranteed delivery, none, some or all of the messages may arrive to ``A2``
.. _Erlang documentation: http://www.erlang.org/faq/academic.html
.. _Erlang documentation: http://www.erlang.org/faq/academic.html

View file

@ -235,7 +235,7 @@ public class UntypedActorDocTestBase {
final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>();
futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout
futures.add(ask(actorB, "reqeest", t)); // using timeout from above
futures.add(ask(actorB, "another request", t)); // using timeout from above
final Future<Iterable<Object>> aggregate = Futures.sequence(futures, system.dispatcher());

View file

@ -24,6 +24,15 @@ import com.typesafe.config.Config;
//#imports-prio-mailbox
//#imports-custom
import akka.dispatch.Envelope;
import akka.dispatch.MessageQueue;
import akka.dispatch.MailboxType;
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
//#imports-custom
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -136,4 +145,32 @@ public class DispatcherDocTestBase {
}
}
//#prio-mailbox
//#mailbox-implementation-example
class MyUnboundedMailbox implements MailboxType {
// This constructor signature must exist, it will be called by Akka
public MyUnboundedMailbox(ActorSystem.Settings settings, Config config) {
// put your initialization code here
}
// The create method is called to create the MessageQueue
public MessageQueue create(Option<ActorRef> owner, Option<ActorSystem> system) {
return new MessageQueue() {
private final Queue<Envelope> queue = new ConcurrentLinkedQueue<Envelope>();
// these must be implemented; queue used as example
public void enqueue(ActorRef receiver, Envelope handle) { queue.offer(handle); }
public Envelope dequeue() { return queue.poll(); }
public int numberOfMessages() { return queue.size(); }
public boolean hasMessages() { return !queue.isEmpty(); }
public void cleanUp(ActorRef owner, MessageQueue deadLetters) {
for (Envelope handle: queue) {
deadLetters.enqueue(owner, handle);
}
}
};
}
}
//#mailbox-implementation-example
}

View file

@ -0,0 +1,399 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.testkit;
import static org.junit.Assert.*;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import com.typesafe.config.ConfigFactory;
import com.typesafe.config.Config;
import akka.actor.ActorKilledException;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Kill;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.dispatch.Await;
import akka.dispatch.Future;
import akka.testkit.CallingThreadDispatcher;
import akka.testkit.TestActor;
import akka.testkit.TestActor.AutoPilot;
import akka.testkit.TestActorRef;
import akka.testkit.JavaTestKit;
import akka.util.Duration;
public class TestKitDocTest {
//#test-actor-ref
static class MyActor extends UntypedActor {
public void onReceive(Object o) throws Exception {
if (o.equals("say42")) {
getSender().tell(42, getSelf());
} else if (o instanceof Exception) {
throw (Exception) o;
}
}
public boolean testMe() { return true; }
}
//#test-actor-ref
private static ActorSystem system;
@BeforeClass
public static void setup() {
final Config config = ConfigFactory.parseString(
"akka.event-handlers = [akka.testkit.TestEventListener]");
system = ActorSystem.create("demoSystem", config);
}
@AfterClass
public static void cleanup() {
system.shutdown();
}
//#test-actor-ref
@Test
public void demonstrateTestActorRef() {
final Props props = new Props(MyActor.class);
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testA");
final MyActor actor = ref.underlyingActor();
assertTrue(actor.testMe());
}
//#test-actor-ref
@Test
public void demonstrateAsk() throws Exception {
//#test-behavior
final Props props = new Props(MyActor.class);
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testB");
final Future<Object> future = akka.pattern.Patterns.ask(ref, "say42", 3000);
assertTrue(future.isCompleted());
assertEquals(42, Await.result(future, Duration.Zero()));
//#test-behavior
}
@Test
public void demonstrateExceptions() {
//#test-expecting-exceptions
final Props props = new Props(MyActor.class);
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "myActor");
try {
ref.receive(new Exception("expected"));
fail("expected an exception to be thrown");
} catch (Exception e) {
assertEquals("expected", e.getMessage());
}
//#test-expecting-exceptions
}
@Test
public void demonstrateWithin() {
//#test-within
new JavaTestKit(system) {{
getRef().tell(42);
new Within(Duration.Zero(), Duration.parse("1 second")) {
// do not put code outside this method, will run afterwards
public void run() {
assertEquals((Integer) 42, expectMsgClass(Integer.class));
}
};
}};
//#test-within
}
@Test
public void demonstrateExpectMsg() {
//#test-expectmsg
new JavaTestKit(system) {{
getRef().tell(42);
final String out = new ExpectMsg<String>("match hint") {
// do not put code outside this method, will run afterwards
protected String match(Object in) {
if (in instanceof Integer) {
return "match";
} else {
throw noMatch();
}
}
}.get(); // this extracts the received message
assertEquals("match", out);
}};
//#test-expectmsg
}
@Test
public void demonstrateReceiveWhile() {
//#test-receivewhile
new JavaTestKit(system) {{
getRef().tell(42);
getRef().tell(43);
getRef().tell("hello");
final String[] out =
new ReceiveWhile<String>(String.class, duration("1 second")) {
// do not put code outside this method, will run afterwards
protected String match(Object in) {
if (in instanceof Integer) {
return in.toString();
} else {
throw noMatch();
}
}
}.get(); // this extracts the received messages
assertArrayEquals(new String[] {"42", "43"}, out);
expectMsgEquals("hello");
}};
//#test-receivewhile
new JavaTestKit(system) {{
//#test-receivewhile-full
new ReceiveWhile<String>( // type of array to be created must match ...
String.class, // ... this class which is needed to that end
duration("100 millis"), // maximum collect time
duration("50 millis"), // maximum time between messages
12 // maximum number of messages to collect
) {
//#match-elided
protected String match(Object in) {
throw noMatch();
}
//#match-elided
};
//#test-receivewhile-full
}};
}
@Test
public void demonstrateAwaitCond() {
//#test-awaitCond
new JavaTestKit(system) {{
getRef().tell(42);
new AwaitCond(
duration("1 second"), // maximum wait time
duration("100 millis") // interval at which to check the condition
) {
// do not put code outside this method, will run afterwards
protected boolean cond() {
// typically used to wait for something to start up
return msgAvailable();
}
};
}};
//#test-awaitCond
}
@Test
@SuppressWarnings("unchecked") // due to generic varargs
public void demonstrateExpect() {
new JavaTestKit(system) {{
getRef().tell("hello");
getRef().tell("hello");
getRef().tell("hello");
getRef().tell("world");
getRef().tell(42);
getRef().tell(42);
//#test-expect
final String hello = expectMsgEquals("hello");
final Object any = expectMsgAnyOf("hello", "world");
final Object[] all = expectMsgAllOf("hello", "world");
final int i = expectMsgClass(Integer.class);
final Number j = expectMsgAnyClassOf(Integer.class, Long.class);
expectNoMsg();
//#test-expect
assertEquals("hello", hello);
assertEquals("hello", any);
assertEquals(42, i);
assertEquals(42, j);
assertArrayEquals(new String[] {"hello", "world"}, all);
}};
}
@Test
public void demonstrateIgnoreMsg() {
//#test-ignoreMsg
new JavaTestKit(system) {{
// ignore all Strings
new IgnoreMsg() {
protected boolean ignore(Object msg) {
return msg instanceof String;
}
};
getRef().tell("hello");
getRef().tell(42);
expectMsgEquals(42);
// remove message filter
ignoreNoMsg();
getRef().tell("hello");
expectMsgEquals("hello");
}};
//#test-ignoreMsg
}
@Test
public void demonstrateDilated() {
//#duration-dilation
new JavaTestKit(system) {{
final Duration original = duration("1 second");
final Duration stretched = dilated(original);
assertTrue("dilated", stretched.gteq(original));
}};
//#duration-dilation
}
@Test
public void demonstrateProbe() {
//#test-probe
// simple actor which just forwards messages
class Forwarder extends UntypedActor {
final ActorRef target;
public Forwarder(ActorRef target) {
this.target = target;
}
public void onReceive(Object msg) {
target.forward(msg, getContext());
}
}
new JavaTestKit(system) {{
// create a test probe
final JavaTestKit probe = new JavaTestKit(system);
// create a forwarder, injecting the probes testActor
final Props props = new Props(new UntypedActorFactory() {
private static final long serialVersionUID = 8927158735963950216L;
public UntypedActor create() {
return new Forwarder(probe.getRef());
}
});
final ActorRef forwarder = system.actorOf(props, "forwarder");
// verify correct forwarding
forwarder.tell(42, getRef());
probe.expectMsgEquals(42);
assertEquals(getRef(), probe.getLastSender());
}};
//#test-probe
}
@Test
public void demonstrateSpecialProbe() {
//#test-special-probe
new JavaTestKit(system) {{
class MyProbe extends JavaTestKit {
public MyProbe() {
super(system);
}
public void assertHello() {
expectMsgEquals("hello");
}
}
final MyProbe probe = new MyProbe();
probe.getRef().tell("hello");
probe.assertHello();
}};
//#test-special-probe
}
@Test
public void demonstrateReply() {
//#test-probe-reply
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
probe.getRef().tell("hello", getRef());
probe.expectMsgEquals("hello");
probe.reply("world");
expectMsgEquals("world");
assertEquals(probe.getRef(), getLastSender());
}};
//#test-probe-reply
}
@Test
public void demonstrateForward() {
//#test-probe-forward
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
probe.getRef().tell("hello", getRef());
probe.expectMsgEquals("hello");
probe.forward(getRef());
expectMsgEquals("hello");
assertEquals(getRef(), getLastSender());
}};
//#test-probe-forward
}
@Test
public void demonstrateWithinProbe() {
try {
//#test-within-probe
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
new Within(duration("1 second")) {
public void run() {
probe.expectMsgEquals("hello");
}
};
}};
//#test-within-probe
} catch (AssertionError e) {
// expected to fail
}
}
@Test
public void demonstrateAutoPilot() {
//#test-auto-pilot
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
// install auto-pilot
probe.setAutoPilot(new TestActor.AutoPilot() {
public AutoPilot run(ActorRef sender, Object msg) {
sender.tell(msg);
return noAutoPilot();
}
});
// first one is replied to directly ...
probe.getRef().tell("hello", getRef());
expectMsgEquals("hello");
// ... but then the auto-pilot switched itself off
probe.getRef().tell("world", getRef());
expectNoMsg();
}};
//#test-auto-pilot
}
// only compilation
public void demonstrateCTD() {
//#calling-thread-dispatcher
system.actorOf(
new Props(MyActor.class)
.withDispatcher(CallingThreadDispatcher.Id()));
//#calling-thread-dispatcher
}
@Test
public void demonstrateEventFilter() {
//#test-event-filter
new JavaTestKit(system) {{
assertEquals("demoSystem", system.name());
final ActorRef victim = system.actorOf(Props.empty(), "victim");
final int result = new EventFilter<Integer>(ActorKilledException.class) {
protected Integer run() {
victim.tell(Kill.getInstance());
return 42;
}
}.from("akka://demoSystem/user/victim").occurrences(1).exec();
assertEquals(42, result);
}};
//#test-event-filter
}
}

View file

@ -0,0 +1,95 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.testkit;
//#fullsample
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.testkit.JavaTestKit;
import akka.util.Duration;
public class TestKitSampleTest {
public static class SomeActor extends UntypedActor {
ActorRef target = null;
public void onReceive(Object msg) {
if (msg.equals("hello")) {
getSender().tell("world");
if (target != null) target.forward(msg, getContext());
} else if (msg instanceof ActorRef) {
target = (ActorRef) msg;
getSender().tell("done");
}
}
}
static ActorSystem system;
@BeforeClass
public static void setup() {
system = ActorSystem.create();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test
public void testIt() {
/*
* Wrap the whole test procedure within a testkit constructor
* if you want to receive actor replies or use Within(), etc.
*/
new JavaTestKit(system) {{
final Props props = new Props(SomeActor.class);
final ActorRef subject = system.actorOf(props);
// can also use JavaTestKit from the outside
final JavaTestKit probe = new JavaTestKit(system);
// inject the probe by passing it to the test subject
// like a real resource would be passed in production
subject.tell(probe.getRef(), getRef());
// await the correct response
expectMsgEquals(duration("1 second"), "done");
// the run() method needs to finish within 3 seconds
new Within(duration("3 seconds")) {
protected void run() {
subject.tell("hello", getRef());
// This is a demo: would normally use expectMsgEquals().
// Wait time is bounded by 3-second deadline above.
new AwaitCond() {
protected boolean cond() {
return probe.msgAvailable();
}
};
// response must have been enqueued to us before probe
expectMsgEquals(Duration.Zero(), "world");
// check that the probe we injected earlier got the msg
probe.expectMsgEquals(Duration.Zero(), "hello");
Assert.assertEquals(getRef(), probe.getLastSender());
// Will wait for the rest of the 3 seconds
expectNoMsg();
}
};
}};
}
}
//#fullsample

View file

@ -92,7 +92,7 @@ There are 4 different types of message dispatchers:
* CallingThreadDispatcher
- This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads,
but it can be used from different threads concurrently for the same actor. See :ref:`TestCallingThreadDispatcherRef`
but it can be used from different threads concurrently for the same actor. See :ref:`Java-CallingThreadDispatcher`
for details and restrictions.
- Sharability: Unlimited
@ -183,3 +183,46 @@ And then an example on how you would use it:
the configuration which describes the dispatcher using this mailbox type; the
mailbox type will be instantiated once for each dispatcher using it.
Creating your own Mailbox type
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
An example is worth a thousand quacks:
.. includecode:: code/docs/dispatcher/DispatcherDocTestBase.java#imports-custom
.. includecode:: code/docs/dispatcher/DispatcherDocTestBase.java#mailbox-implementation-example
And then you just specify the FQCN of your MailboxType as the value of the "mailbox-type" in the dispatcher configuration.
.. note::
Make sure to include a constructor which takes
``akka.actor.ActorSystem.Settings`` and ``com.typesafe.config.Config``
arguments, as this constructor is invoked reflectively to construct your
mailbox type. The config passed in as second argument is that section from
the configuration which describes the dispatcher using this mailbox type; the
mailbox type will be instantiated once for each dispatcher using it.
Special Semantics of ``system.actorOf``
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
In order to make ``system.actorOf`` both synchronous and non-blocking while
keeping the return type :class:`ActorRef` (and the semantics that the returned
ref is fully functional), special handling takes place for this case. Behind
the scenes, a hollow kind of actor reference is constructed, which is sent to
the systems guardian actor who actually creates the actor and its context and
puts those inside the reference. Until that has happened, messages sent to the
:class:`ActorRef` will be queued locally, and only upon swapping the real
filling in will they be transferred into the real mailbox. Thus,
.. code-block:: scala
final Props props = ...
// this actor uses MyCustomMailbox, which is assumed to be a singleton
system.actorOf(props.withDispatcher("myCustomMailbox").tell("bang");
assert(MyCustomMailbox.getInstance().getLastEnqueued().equals("bang"));
will probably fail; you will have to allow for some time to pass and retry the
check à la :meth:`TestKit.awaitCond`.

View file

@ -4,11 +4,574 @@
Testing Actor Systems (Java)
##############################
Due to the conciseness of test DSLs available for Scala, it may be a good idea
to write the test suite in that language even if the main project is written in
Java. If that is not desirable, you can also use :class:`TestKit` and friends
from Java, albeit with more verbose syntax Munish Gupta has `published a nice
post <http://www.akkaessentials.in/2012/05/using-testkit-with-java.html>`_
showing several patterns you may find useful, and for reference documentation
please refer to :ref:`akka-testkit` until that section has been ported over to
cover Java in full.
As with any piece of software, automated tests are a very important part of the
development cycle. The actor model presents a different view on how units of
code are delimited and how they interact, which has an influence on how to
perform tests.
.. note::
Due to the conciseness of test DSLs available for Scala (`ScalaTest`_,
`Specs2`_, `ScalaCheck`_), it may be a good idea to write the test suite in
that language even if the main project is written in Java. If that is not
desirable, you can also use :class:`TestKit` and friends from Java, albeit
with more verbose syntax which is covered below. Munish Gupta has `published
a nice post
<http://www.akkaessentials.in/2012/05/using-testkit-with-java.html>`_ showing
several patterns you may find useful.
.. _ScalaTest: http://scalatest.org/
.. _Specs2: http://specs2.org/
.. _ScalaCheck: http://code.google.com/p/scalacheck/
Akka comes with a dedicated module :mod:`akka-testkit` for supporting tests at
different levels, which fall into two clearly distinct categories:
- Testing isolated pieces of code without involving the actor model, meaning
without multiple threads; this implies completely deterministic behavior
concerning the ordering of events and no concurrency concerns and will be
called **Unit Testing** in the following.
- Testing (multiple) encapsulated actors including multi-threaded scheduling;
this implies non-deterministic order of events but shielding from
concurrency concerns by the actor model and will be called **Integration
Testing** in the following.
There are of course variations on the granularity of tests in both categories,
where unit testing reaches down to white-box tests and integration testing can
encompass functional tests of complete actor networks. The important
distinction lies in whether concurrency concerns are part of the test or not.
The tools offered are described in detail in the following sections.
.. note::
Be sure to add the module :mod:`akka-testkit` to your dependencies.
Unit Testing with :class:`TestActorRef`
=======================================
Testing the business logic inside :class:`Actor` classes can be divided into
two parts: first, each atomic operation must work in isolation, then sequences
of incoming events must be processed correctly, even in the presence of some
possible variability in the ordering of events. The former is the primary use
case for single-threaded unit testing, while the latter can only be verified in
integration tests.
Normally, the :class:`ActorRef` shields the underlying :class:`Actor` instance
from the outside, the only communications channel is the actor's mailbox. This
restriction is an impediment to unit testing, which led to the inception of the
:class:`TestActorRef`. This special type of reference is designed specifically
for test purposes and allows access to the actor in two ways: either by
obtaining a reference to the underlying actor instance, or by invoking or
querying the actor's behaviour (:meth:`receive`). Each one warrants its own
section below.
Obtaining a Reference to an :class:`Actor`
------------------------------------------
Having access to the actual :class:`Actor` object allows application of all
traditional unit testing techniques on the contained methods. Obtaining a
reference is done like this:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-actor-ref
Since :class:`TestActorRef` is generic in the actor type it returns the
underlying actor with its proper static type. From this point on you may bring
any unit testing tool to bear on your actor as usual.
Testing the Actor's Behavior
----------------------------
When the dispatcher invokes the processing behavior of an actor on a message,
it actually calls :meth:`apply` on the current behavior registered for the
actor. This starts out with the return value of the declared :meth:`receive`
method, but it may also be changed using :meth:`become` and :meth:`unbecome` in
response to external messages. All of this contributes to the overall actor
behavior and it does not lend itself to easy testing on the :class:`Actor`
itself. Therefore the :class:`TestActorRef` offers a different mode of
operation to complement the :class:`Actor` testing: it supports all operations
also valid on normal :class:`ActorRef`. Messages sent to the actor are
processed synchronously on the current thread and answers may be sent back as
usual. This trick is made possible by the :class:`CallingThreadDispatcher`
described below (see `CallingThreadDispatcher`_); this dispatcher is set
implicitly for any actor instantiated into a :class:`TestActorRef`.
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-behavior
As the :class:`TestActorRef` is a subclass of :class:`LocalActorRef` with a few
special extras, also aspects like supervision and restarting work properly, but
beware that execution is only strictly synchronous as long as all actors
involved use the :class:`CallingThreadDispatcher`. As soon as you add elements
which include more sophisticated scheduling you leave the realm of unit testing
as you then need to think about asynchronicity again (in most cases the problem
will be to wait until the desired effect had a chance to happen).
One more special aspect which is overridden for single-threaded tests is the
:meth:`receiveTimeout`, as including that would entail asynchronous queuing of
:obj:`ReceiveTimeout` messages, violating the synchronous contract.
.. note::
To summarize: :class:`TestActorRef` overwrites two fields: it sets the
dispatcher to :obj:`CallingThreadDispatcher.global` and it sets the
:obj:`receiveTimeout` to None.
The Way In-Between: Expecting Exceptions
----------------------------------------
If you want to test the actor behavior, including hotswapping, but without
involving a dispatcher and without having the :class:`TestActorRef` swallow
any thrown exceptions, then there is another mode available for you: just use
the :meth:`receive` method :class:`TestActorRef`, which will be forwarded to the
underlying actor:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-expecting-exceptions
Use Cases
---------
You may of course mix and match both modi operandi of :class:`TestActorRef` as
suits your test needs:
- one common use case is setting up the actor into a specific internal state
before sending the test message
- another is to verify correct internal state transitions after having sent
the test message
Feel free to experiment with the possibilities, and if you find useful
patterns, don't hesitate to let the Akka forums know about them! Who knows,
common operations might even be worked into nice DSLs.
Integration Testing with :class:`JavaTestKit`
=============================================
When you are reasonably sure that your actor's business logic is correct, the
next step is verifying that it works correctly within its intended environment.
The definition of the environment depends of course very much on the problem at
hand and the level at which you intend to test, ranging for
functional/integration tests to full system tests. The minimal setup consists
of the test procedure, which provides the desired stimuli, the actor under
test, and an actor receiving replies. Bigger systems replace the actor under
test with a network of actors, apply stimuli at varying injection points and
arrange results to be sent from different emission points, but the basic
principle stays the same in that a single procedure drives the test.
The :class:`JavaTestKit` class contains a collection of tools which makes this
common task easy.
.. includecode:: code/docs/testkit/TestKitSampleTest.java#fullsample
The :class:`JavaTestKit` contains an actor named :obj:`testActor` which is the
entry point for messages to be examined with the various ``expectMsg...``
assertions detailed below. The test actors reference is obtained using the
:meth:`getRef()` method as demonstrated above. The :obj:`testActor` may also
be passed to other actors as usual, usually subscribing it as notification
listener. There is a whole set of examination methods, e.g. receiving all
consecutive messages matching certain criteria, receiving a whole sequence of
fixed messages or classes, receiving nothing for some time, etc.
The ActorSystem passed in to the constructor of JavaTestKit is accessible via the
:meth:`getSystem()` method.
.. note::
Remember to shut down the actor system after the test is finished (also in
case of failure) so that all actors—including the test actor—are stopped.
Built-In Assertions
-------------------
The above mentioned :meth:`expectMsgEquals` is not the only method for
formulating assertions concerning received messages, the full set is this:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-expect
In these examples, the maximum durations you will find mentioned below are left
out, in which case they use the default value from configuration item
``akka.test.single-expect-default`` which itself defaults to 3 seconds (or they
obey the innermost enclosing :class:`Within` as detailed :ref:`below
<JavaTestKit.within>`). The full signatures are:
* :meth:`public <T> T expectMsgEquals(Duration max, T msg)`
The given message object must be received within the specified time; the
object will be returned.
* :meth:`public Object expectMsgAnyOf(Duration max, Object... msg)`
An object must be received within the given time, and it must be equal
(compared with ``equals()``) to at least one of the passed reference
objects; the received object will be returned.
* :meth:`public Object[] expectMsgAllOf(Duration max, Object... msg)`
A number of objects matching the size of the supplied object array must be
received within the given time, and for each of the given objects there
must exist at least one among the received ones which equals it (compared
with ``equals()``). The full sequence of received objects is returned in
the order received.
* :meth:`public <T> T expectMsgClass(Duration max, Class<T> c)`
An object which is an instance of the given :class:`Class` must be received
within the allotted time frame; the object will be returned. Note that this
does a conformance check, if you need the class to be equal you need to
verify that afterwards.
* :meth:`public <T> T expectMsgAnyClassOf(Duration max, Class<? extends T>... c)`
An object must be received within the given time, and it must be an
instance of at least one of the supplied :class:`Class` objects; the
received object will be returned. Note that this does a conformance check,
if you need the class to be equal you need to verify that afterwards.
.. note::
Because of a limitation in Javas type system it may be necessary to add
``@SuppressWarnings("unchecked")`` when using this method.
* :meth:`public void expectNoMsg(Duration max)`
No message must be received within the given time. This also fails if a
message has been received before calling this method which has not been
removed from the queue using one of the other methods.
For cases which require more refined conditions there are constructs which take
code blocks:
* **ExpectMsg<T>**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-expectmsg
The :meth:`match(Object in)` method will be evaluated once a message has
been received within the allotted time (which may be given as constructor
argument). If it throws ``noMatch()`` (where it is sufficient to call that
method; the ``throw`` keyword is only needed in cases where the compiler
would otherwise complain about wrong return types—Java is lacking Scalas
notion of a type which signifies “will not ever return normally”), then the
expectation fails with an :class:`AssertionError`, otherwise the matched
and possibly transformed object is stored for retrieval using the
:meth:`get()` method.
* **ReceiveWhile<T>**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-receivewhile
This construct works like ExpectMsg, but it continually collects messages
as long as they match the criteria, and it does not fail when a
non-matching one is encountered. Collecting messages also ends when the
time is up, when too much time passes between messages or when enough
messages have been received.
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-receivewhile-full
:exclude: match-elided
The need to specify the ``String`` result type twice results from the need
to create a correctly typed array and Javas inability to infer the classs
type argument.
* **AwaitCond**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-awaitCond
This general construct is not connected with the test kits message
reception, the embedded condition can compute the boolean result from
anything in scope.
There are also cases where not all messages sent to the test kit are actually
relevant to the test, but removing them would mean altering the actors under
test. For this purpose it is possible to ignore certain messages:
* **IgnoreMsg**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-ignoreMsg
Expecting Log Messages
----------------------
Since an integration test does not allow to the internal processing of the
participating actors, verifying expected exceptions cannot be done directly.
Instead, use the logging system for this purpose: replacing the normal event
handler with the :class:`TestEventListener` and using an :class:`EventFilter`
allows assertions on log messages, including those which are generated by
exceptions:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-event-filter
If a number of occurrences is specific—as demonstrated above—then ``exec()``
will block until that number of matching messages have been received or the
timeout configured in ``akka.test.filter-leeway`` is used up (time starts
counting after the ``run()`` method returns). In case of a timeout the test
fails.
.. note::
Be sure to exchange the default event handler with the
:class:`TestEventListener` in your ``application.conf`` to enable this
function::
akka.event-handlers = [akka.testkit.TestEventListener]
.. _JavaTestKit.within:
Timing Assertions
-----------------
Another important part of functional testing concerns timing: certain events
must not happen immediately (like a timer), others need to happen before a
deadline. Therefore, all examination methods accept an upper time limit within
the positive or negative result must be obtained. Lower time limits need to be
checked external to the examination, which is facilitated by a new construct
for managing time constraints:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-within
The block in :meth:`Within.run()` must complete after a :ref:`Duration` which
is between :obj:`min` and :obj:`max`, where the former defaults to zero. The
deadline calculated by adding the :obj:`max` parameter to the block's start
time is implicitly available within the block to all examination methods, if
you do not specify it, it is inherited from the innermost enclosing
:meth:`within` block.
It should be noted that if the last message-receiving assertion of the block is
:meth:`expectNoMsg` or :meth:`receiveWhile`, the final check of the
:meth:`within` is skipped in order to avoid false positives due to wake-up
latencies. This means that while individual contained assertions still use the
maximum time bound, the overall block may take arbitrarily longer in this case.
.. note::
All times are measured using ``System.nanoTime``, meaning that they describe
wall time, not CPU time or system time.
Accounting for Slow Test Systems
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The tight timeouts you use during testing on your lightning-fast notebook will
invariably lead to spurious test failures on the heavily loaded Jenkins server
(or similar). To account for this situation, all maximum durations are
internally scaled by a factor taken from the :ref:`configuration`,
``akka.test.timefactor``, which defaults to 1.
You can scale other durations with the same factor by using the implicit conversion
in ``akka.testkit`` package object to add dilated function to :class:`Duration`.
.. includecode:: code/docs/testkit/TestKitDocTest.java#duration-dilation
Using Multiple Probe Actors
---------------------------
When the actors under test are supposed to send various messages to different
destinations, it may be difficult distinguishing the message streams arriving
at the :obj:`testActor` when using the :class:`JavaTestKit` as shown until now.
Another approach is to use it for creation of simple probe actors to be
inserted in the message flows. The functionality is best explained using a
small example:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-probe
This simple test verifies an equally simple Forwarder actor by injecting a
probe as the forwarders target. Another example would be two actors A and B
which collaborate by A sending messages to B. In order to verify this message
flow, a :class:`TestProbe` could be inserted as target of A, using the
forwarding capabilities or auto-pilot described below to include a real B in
the test setup.
Probes may also be equipped with custom assertions to make your test code even
more concise and clear:
.. includecode:: code/docs/testkit/TestKitDocTest.java
:include: test-special-probe
You have complete flexibility here in mixing and matching the
:class:`JavaTestKit` facilities with your own checks and choosing an intuitive
name for it. In real life your code will probably be a bit more complicated
than the example given above; just use the power!
Replying to Messages Received by Probes
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The probe stores the sender of the last dequeued message (i.e. after its
``expectMsg*`` reception), which may be retrieved using the
:meth:`getLastSender()` method. This information can also implicitly be used
for having the probe reply to the last received message:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-probe-reply
Forwarding Messages Received by Probes
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The probe can also forward a received message (i.e. after its ``expectMsg*``
reception), retaining the original sender:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-probe-forward
Auto-Pilot
^^^^^^^^^^
Receiving messages in a queue for later inspection is nice, but in order to
keep a test running and verify traces later you can also install an
:class:`AutoPilot` in the participating test probes (actually in any
:class:`TestKit`) which is invoked before enqueueing to the inspection queue.
This code can be used to forward messages, e.g. in a chain ``A --> Probe -->
B``, as long as a certain protocol is obeyed.
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-auto-pilot
The :meth:`run` method must return the auto-pilot for the next message, wrapped
in an :class:`Option`; setting it to :obj:`None` terminates the auto-pilot.
Caution about Timing Assertions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The behavior of :meth:`within` blocks when using test probes might be perceived
as counter-intuitive: you need to remember that the nicely scoped deadline as
described :ref:`above <JavaTestKit.within>` is local to each probe. Hence, probes
do not react to each other's deadlines or to the deadline set in an enclosing
:class:`JavaTestKit` instance:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-within-probe
Here, the ``expectMsgEquals`` call will use the default timeout.
.. _Java-CallingThreadDispatcher:
CallingThreadDispatcher
=======================
The :class:`CallingThreadDispatcher` serves good purposes in unit testing, as
described above, but originally it was conceived in order to allow contiguous
stack traces to be generated in case of an error. As this special dispatcher
runs everything which would normally be queued directly on the current thread,
the full history of a message's processing chain is recorded on the call stack,
so long as all intervening actors run on this dispatcher.
How to use it
-------------
Just set the dispatcher as you normally would:
.. includecode:: code/docs/testkit/TestKitDocTest.java#calling-thread-dispatcher
How it works
------------
When receiving an invocation, the :class:`CallingThreadDispatcher` checks
whether the receiving actor is already active on the current thread. The
simplest example for this situation is an actor which sends a message to
itself. In this case, processing cannot continue immediately as that would
violate the actor model, so the invocation is queued and will be processed when
the active invocation on that actor finishes its processing; thus, it will be
processed on the calling thread, but simply after the actor finishes its
previous work. In the other case, the invocation is simply processed
immediately on the current thread. Futures scheduled via this dispatcher are
also executed immediately.
This scheme makes the :class:`CallingThreadDispatcher` work like a general
purpose dispatcher for any actors which never block on external events.
In the presence of multiple threads it may happen that two invocations of an
actor running on this dispatcher happen on two different threads at the same
time. In this case, both will be processed directly on their respective
threads, where both compete for the actor's lock and the loser has to wait.
Thus, the actor model is left intact, but the price is loss of concurrency due
to limited scheduling. In a sense this is equivalent to traditional mutex style
concurrency.
The other remaining difficulty is correct handling of suspend and resume: when
an actor is suspended, subsequent invocations will be queued in thread-local
queues (the same ones used for queuing in the normal case). The call to
:meth:`resume`, however, is done by one specific thread, and all other threads
in the system will probably not be executing this specific actor, which leads
to the problem that the thread-local queues cannot be emptied by their native
threads. Hence, the thread calling :meth:`resume` will collect all currently
queued invocations from all threads into its own queue and process them.
Limitations
-----------
If an actor's behavior blocks on a something which would normally be affected
by the calling actor after having sent the message, this will obviously
dead-lock when using this dispatcher. This is a common scenario in actor tests
based on :class:`CountDownLatch` for synchronization:
.. code-block:: scala
val latch = new CountDownLatch(1)
actor ! startWorkAfter(latch) // actor will call latch.await() before proceeding
doSomeSetupStuff()
latch.countDown()
The example would hang indefinitely within the message processing initiated on
the second line and never reach the fourth line, which would unblock it on a
normal dispatcher.
Thus, keep in mind that the :class:`CallingThreadDispatcher` is not a
general-purpose replacement for the normal dispatchers. On the other hand it
may be quite useful to run your actor network on it for testing, because if it
runs without dead-locking chances are very high that it will not dead-lock in
production.
.. warning::
The above sentence is unfortunately not a strong guarantee, because your
code might directly or indirectly change its behavior when running on a
different dispatcher. If you are looking for a tool to help you debug
dead-locks, the :class:`CallingThreadDispatcher` may help with certain error
scenarios, but keep in mind that it has may give false negatives as well as
false positives.
Benefits
--------
To summarize, these are the features with the :class:`CallingThreadDispatcher`
has to offer:
- Deterministic execution of single-threaded tests while retaining nearly full
actor semantics
- Full message processing history leading up to the point of failure in
exception stack traces
- Exclusion of certain classes of dead-lock scenarios
.. _actor.logging-java:
Tracing Actor Invocations
=========================
The testing facilities described up to this point were aiming at formulating
assertions about a systems behavior. If a test fails, it is usually your job
to find the cause, fix it and verify the test again. This process is supported
by debuggers as well as logging, where the Akka toolkit offers the following
options:
* *Logging of exceptions thrown within Actor instances*
This is always on; in contrast to the other logging mechanisms, this logs at
``ERROR`` level.
* *Logging of special messages*
Actors handle certain special messages automatically, e.g. :obj:`Kill`,
:obj:`PoisonPill`, etc. Tracing of these message invocations is enabled by
the setting ``akka.actor.debug.autoreceive``, which enables this on all
actors.
* *Logging of the actor lifecycle*
Actor creation, start, restart, monitor start, monitor stop and stop may be traced by
enabling the setting ``akka.actor.debug.lifecycle``; this, too, is enabled
uniformly on all actors.
All these messages are logged at ``DEBUG`` level. To summarize, you can enable
full logging of actor activities using this configuration fragment::
akka {
loglevel = DEBUG
actor {
debug {
autoreceive = on
lifecycle = on
}
}
}

View file

@ -82,13 +82,6 @@ that is used in log messages and for identifying actors. The name must not be em
or start with ``$``. If the given name is already in use by another child to the
same parent actor an `InvalidActorNameException` is thrown.
.. warning::
Creating top-level actors with ``system.actorOf`` is a blocking operation,
hence it may dead-lock due to starvation if the default dispatcher is
overloaded. To avoid problems, do not call this method from within actors or
futures which run on the default dispatcher.
Actors are automatically started asynchronously when created.
When you create the ``UntypedActor`` then it will automatically call the ``preStart``
callback method on the ``UntypedActor`` class. This is an excellent place to

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