Merging in master, huge work trying to get things to compile, tests not green at this stage

This commit is contained in:
Viktor Klang 2012-07-06 17:04:04 +02:00
commit ac5b5de90a
68 changed files with 3759 additions and 2144 deletions

1
.gitignore vendored
View file

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

View file

@ -321,7 +321,7 @@ public class JavaFutureTests {
Future<Object> f = p.future().recoverWith(new Recover<Future<Object>>() { Future<Object> f = p.future().recoverWith(new Recover<Future<Object>>() {
public Future<Object> recover(Throwable t) throws Throwable { public Future<Object> recover(Throwable t) throws Throwable {
if (t == fail) if (t == fail)
return Futures.<Object> successful("foo", system.dispatcher()).future(); return Futures.<Object> successful("foo", system.dispatcher());
else else
throw t; throw t;
} }

View file

@ -167,7 +167,7 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
system.scheduler.scheduleOnce(200 millis) { system.shutdown() } system.scheduler.scheduleOnce(200 millis) { system.shutdown() }
var failing = false var failing = false
var created = Vector.empty[ActorRef] var created = Vector.empty[ActorRef]
while (!system.isTerminated && system.uptime < 5) { while (!system.isTerminated) {
try { try {
val t = system.actorOf(Props[ActorSystemSpec.Terminater]) val t = system.actorOf(Props[ActorSystemSpec.Terminater])
failing must not be true // because once failing => always failing (its due to shutdown) failing must not be true // because once failing => always failing (its due to shutdown)
@ -175,12 +175,14 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
} catch { } catch {
case _: IllegalStateException failing = true case _: IllegalStateException failing = true
} }
if (!failing && system.uptime >= 5) {
println(created.last)
println(system.asInstanceOf[ExtendedActorSystem].printTree)
fail("System didn't terminate within 5 seconds")
}
} }
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()) created filter (ref !ref.isTerminated && !ref.asInstanceOf[ActorRefWithCell].underlying.isInstanceOf[UnstartedCell]) must be(Seq())
} }

View file

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

View file

@ -376,8 +376,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
val child = context.watch(context.actorOf(Props(new Actor { val child = context.watch(context.actorOf(Props(new Actor {
override def postRestart(reason: Throwable): Unit = testActor ! "child restarted" override def postRestart(reason: Throwable): Unit = testActor ! "child restarted"
def receive = { def receive = {
case "die" throw new IllegalStateException("OHNOES") case l: TestLatch Await.ready(l, 5 seconds); throw new IllegalStateException("OHNOES")
case "test" sender ! "child green" case "test" sender ! "child green"
} }
}), "child")) }), "child"))
@ -385,14 +385,18 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
def receive = { def receive = {
case t @ Terminated(`child`) testActor ! "child terminated" case t @ Terminated(`child`) testActor ! "child terminated"
case "die" child ! "die" case l: TestLatch child ! l
case "test" sender ! "green" case "test" sender ! "green"
case "testchild" child forward "test" case "testchild" child forward "test"
} }
})) }))
parent ! "die" val latch = TestLatch()
parent ! latch
parent ! "testchild" parent ! "testchild"
EventFilter[IllegalStateException]("OHNOES", occurrences = 2) intercept {
latch.countDown()
}
expectMsg("parent restarted") expectMsg("parent restarted")
expectMsg("child terminated") expectMsg("child terminated")
parent ! "test" parent ! "test"

View file

@ -803,7 +803,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
flow { y << 2 } flow { y << 2 }
assert(Await.result(z, timeout.duration) === 42) assert(Await.result(z, timeout.duration) === 42)
} }*/
"run callbacks async" in { "run callbacks async" in {
val latch = Vector.fill(10)(new TestLatch) val latch = Vector.fill(10)(new TestLatch)

View file

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

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

@ -15,7 +15,8 @@ import akka.AkkaException
* class ActorWithProtocol extends Actor with Stash { * class ActorWithProtocol extends Actor with Stash {
* def receive = { * def receive = {
* case "open" * case "open"
* unstashAll { * unstashAll()
* context.become {
* case "write" // do writing... * case "write" // do writing...
* case "close" * case "close"
* unstashAll() * unstashAll()

View file

@ -29,12 +29,12 @@ object Futures {
/** /**
* Java API, creates an already completed Promise with the specified exception * Java API, creates an already completed Promise with the specified exception
*/ */
def failed[T](exception: Throwable, executor: ExecutionContext): Promise[T] = Promise.failed(exception) def failed[T](exception: Throwable, executor: ExecutionContext): Future[T] = Promise.failed(exception).future
/** /**
* Java API, Creates an already completed Promise with the specified result * Java API, Creates an already completed Promise with the specified result
*/ */
def successful[T](result: T, executor: ExecutionContext): Promise[T] = Promise.successful(result) def successful[T](result: T, executor: ExecutionContext): Future[T] = Promise.successful(result).future
/** /**
* Java API. * Java API.

View file

@ -592,6 +592,7 @@ object Logging {
/** Null Object used for errors without cause Throwable */ /** Null Object used for errors without cause Throwable */
object NoCause extends NoStackTrace object NoCause extends NoStackTrace
} }
def noCause = Error.NoCause
/** /**
* For WARNING Logging * For WARNING Logging

View file

@ -8,6 +8,7 @@ import language.implicitConversions
import scala.Some import scala.Some
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.util.control.NoStackTrace
/** /**
* A Function interface. Used to create first-class-functions is Java. * A Function interface. Used to create first-class-functions is Java.
@ -47,6 +48,76 @@ trait Creator[T] {
def create(): 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> * This class represents optional values. Instances of <code>Option</code>
* are either instances of case class <code>Some</code> or it is case * are either instances of case class <code>Some</code> or it is case
@ -116,8 +187,9 @@ object Option {
* This class hold common utilities for Java * This class hold common utilities for Java
*/ */
object Util { object Util {
/**
* Given a Class returns a Scala Manifest of that Class
*/
def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz) def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
def arrayToSeq[T](arr: Array[T]): Seq[T] = arr.toSeq
def arrayToSeq(classes: Array[Class[_]]): Seq[Class[_]] = classes.toSeq
} }

View file

@ -6,11 +6,10 @@ package akka.agent
import akka.actor._ import akka.actor._
import akka.japi.{ Function JFunc, Procedure JProc } import akka.japi.{ Function JFunc, Procedure JProc }
import akka.dispatch._
import akka.pattern.ask import akka.pattern.ask
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.stm._ import scala.concurrent.stm._
import scala.concurrent.Await import scala.concurrent.{ Future, Promise, Await }
/** /**
* Used internally to send functions. * Used internally to send functions.
@ -128,9 +127,9 @@ class Agent[T](initialValue: T, system: ActorSystem) {
def dispatch = ask(updater, Alter(f))(timeout).asInstanceOf[Future[T]] def dispatch = ask(updater, Alter(f))(timeout).asInstanceOf[Future[T]]
val txn = Txn.findCurrent val txn = Txn.findCurrent
if (txn.isDefined) { if (txn.isDefined) {
val result = Promise[T]()(system.dispatcher) val result = Promise[T]()
Txn.afterCommit(status result completeWith dispatch)(txn.get) Txn.afterCommit(status result completeWith dispatch)(txn.get)
result result.future
} else dispatch } else dispatch
} }
@ -169,14 +168,14 @@ class Agent[T](initialValue: T, system: ActorSystem) {
* still be executed in order. * still be executed in order.
*/ */
def alterOff(f: T T)(timeout: Timeout): Future[T] = { def alterOff(f: T T)(timeout: Timeout): Future[T] = {
val result = Promise[T]()(system.dispatcher) val result = Promise[T]()
send((value: T) { send((value: T) {
suspend() suspend()
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this, ref)).withDispatcher("akka.agent.alter-off-dispatcher")) val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this, ref)).withDispatcher("akka.agent.alter-off-dispatcher"))
result completeWith ask(threadBased, Alter(f))(timeout).asInstanceOf[Future[T]] result completeWith ask(threadBased, Alter(f))(timeout).asInstanceOf[Future[T]]
value value
}) })
result result.future
} }
/** /**

View file

@ -9,8 +9,6 @@ akka {
cluster { cluster {
# Initial contact points of the cluster. Nodes to join at startup if auto-join = on. # 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" # 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. # Leave as empty if the node should be a singleton cluster.
seed-nodes = [] seed-nodes = []
@ -27,12 +25,6 @@ akka {
# network partition. # network partition.
auto-down = off auto-down = off
# the number of gossip daemon actors
nr-of-gossip-daemons = 4
# the number of deputy nodes (the nodes responsible for breaking network partitions)
nr-of-deputy-nodes = 3
# how long should the node wait before starting the periodic tasks maintenance tasks? # how long should the node wait before starting the periodic tasks maintenance tasks?
periodic-tasks-initial-delay = 1s periodic-tasks-initial-delay = 1s
@ -48,15 +40,27 @@ akka {
# how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring? # how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring?
unreachable-nodes-reaper-interval = 1s unreachable-nodes-reaper-interval = 1s
# How often the current state (Gossip) should be published for reading from the outside.
# A value of 0 s can be used to always publish the state, when it happens.
publish-state-interval = 1s
# A joining node stops sending heartbeats to the node to join if it hasn't become member # A joining node stops sending heartbeats to the node to join if it hasn't become member
# of the cluster within this deadline. # of the cluster within this deadline.
join-timeout = 60s join-timeout = 60s
# The id of the dispatcher to use for cluster actors. If not specified default dispatcher is used.
# If specified you need to define the settings of the actual dispatcher.
use-dispatcher = ""
# Gossip to random node with newer or older state information, if any with some # Gossip to random node with newer or older state information, if any with some
# this probability. Otherwise Gossip to any random live node. # 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. # Probability value is between 0.0 and 1.0. 0.0 means never, 1.0 means always.
gossip-different-view-probability = 0.8 gossip-different-view-probability = 0.8
# Limit number of merge conflicts per second that are handled. If the limit is
# exceeded the conflicting gossip messages are dropped and will reappear later.
max-gossip-merge-rate = 5.0
failure-detector { failure-detector {
# defines the failure detector threshold # defines the failure detector threshold
@ -93,5 +97,13 @@ akka {
tick-duration = 33ms tick-duration = 33ms
ticks-per-wheel = 512 ticks-per-wheel = 512
} }
# Netty blocks when sending to broken connections, and this circuit breaker
# is used to reduce connect attempts to broken connections.
send-circuit-breaker {
max-failures = 3
call-timeout = 2 s
reset-timeout = 30 s
}
} }
} }

View file

@ -167,8 +167,9 @@ class AccrualFailureDetector(
val φ = phi(timeDiff, mean + acceptableHeartbeatPauseMillis, stdDeviation) val φ = phi(timeDiff, mean + acceptableHeartbeatPauseMillis, stdDeviation)
// FIXME change to debug log level, when failure detector is stable // FIXME change to debug log level, when failure detector is stable
if (φ > 1.0) log.info("Phi value [{}] for connection [{}], after [{} ms], based on [{}]", if (φ > 1.0 && timeDiff < (acceptableHeartbeatPauseMillis + 5000))
φ, connection, timeDiff, "N(" + mean + ", " + stdDeviation + ")") log.info("Phi value [{}] for connection [{}], after [{} ms], based on [{}]",
φ, connection, timeDiff, "N(" + mean + ", " + stdDeviation + ")")
φ φ
} }

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,905 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import scala.concurrent.util.{ Deadline, Duration }
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.{ Actor, ActorLogging, ActorRef, Address, Cancellable, Props, RootActorPath, PoisonPill, Scheduler }
import akka.actor.Status.Failure
import akka.routing.ScatterGatherFirstCompletedRouter
import akka.util.Timeout
import akka.pattern.{ AskTimeoutException, ask, pipe }
import MemberStatus._
/**
* Base trait for all cluster messages. All ClusterMessage's are serializable.
*
* FIXME Protobuf all ClusterMessages
*/
trait ClusterMessage extends Serializable
/**
* Cluster commands sent by the USER.
*/
object ClusterUserAction {
/**
* Command to join the cluster. Sent when a node (represented by 'address')
* wants to join another node (the receiver).
*/
case class Join(address: Address) extends ClusterMessage
/**
* Command to leave the cluster.
*/
case class Leave(address: Address) extends ClusterMessage
/**
* Command to mark node as temporary down.
*/
case class Down(address: Address) extends ClusterMessage
}
/**
* INTERNAL API
*/
private[cluster] object InternalClusterAction {
/**
* Command to initiate join another node (represented by 'address').
* Join will be sent to the other node.
*/
case class JoinTo(address: Address) extends ClusterMessage
/**
* Start message of the process to join one of the seed nodes.
* The node sends `InitJoin` to all seed nodes, which replies
* with `InitJoinAck`. The first reply is used others are discarded.
* The node sends `Join` command to the seed node that replied first.
*/
case object JoinSeedNode extends ClusterMessage
/**
* @see JoinSeedNode
*/
case object InitJoin extends ClusterMessage
/**
* @see JoinSeedNode
*/
case class InitJoinAck(address: Address) extends ClusterMessage
case object GossipTick
case object HeartbeatTick
case object ReapUnreachableTick
case object LeaderActionsTick
case object PublishStateTick
case class SendClusterMessage(to: Address, msg: ClusterMessage)
case class SendGossipTo(address: Address)
case object GetClusterCoreRef
case class Ping(timestamp: Long = System.currentTimeMillis) extends ClusterMessage
case class Pong(ping: Ping, timestamp: Long = System.currentTimeMillis) extends ClusterMessage
}
/**
* INTERNAL API.
*
* Cluster commands sent by the LEADER.
*/
private[cluster] object ClusterLeaderAction {
/**
* Command to mark a node to be removed from the cluster immediately.
* Can only be sent by the leader.
*/
case class Exit(address: Address) extends ClusterMessage
/**
* Command to remove a node from the cluster immediately.
*/
case class Remove(address: Address) extends ClusterMessage
}
/**
* INTERNAL API
*
* The contextual pieces that ClusterDaemon actors need.
* Makes it easier to test the actors without using the Cluster extension.
*/
private[cluster] trait ClusterEnvironment {
private[cluster] def settings: ClusterSettings
private[cluster] def failureDetector: FailureDetector
private[cluster] def selfAddress: Address
private[cluster] def scheduler: Scheduler
private[cluster] def seedNodes: IndexedSeq[Address]
private[cluster] def notifyMembershipChangeListeners(members: SortedSet[Member]): Unit
private[cluster] def publishLatestGossip(gossip: Gossip): Unit
private[cluster] def publishLatestStats(stats: ClusterStats): Unit
private[cluster] def shutdown(): Unit
}
/**
* INTERNAL API.
*
* Supervisor managing the different Cluster daemons.
*/
private[cluster] final class ClusterDaemon(environment: ClusterEnvironment) extends Actor with ActorLogging {
val configuredDispatcher = environment.settings.UseDispatcher
val core = context.actorOf(Props(new ClusterCoreDaemon(environment)).
withDispatcher(configuredDispatcher), name = "core")
val heartbeat = context.actorOf(Props(new ClusterHeartbeatDaemon(environment)).
withDispatcher(configuredDispatcher), name = "heartbeat")
def receive = {
case InternalClusterAction.GetClusterCoreRef sender ! core
}
}
/**
* INTERNAL API.
*/
private[cluster] final class ClusterCoreDaemon(environment: ClusterEnvironment) extends Actor with ActorLogging {
import ClusterLeaderAction._
import InternalClusterAction._
import ClusterHeartbeatSender._
def selfAddress = environment.selfAddress
def clusterScheduler = environment.scheduler
def failureDetector = environment.failureDetector
val settings = environment.settings
import settings._
val vclockNode = VectorClock.Node(selfAddress.toString)
val selfHeartbeat = Heartbeat(selfAddress)
// note that self is not initially member,
// and the Gossip is not versioned for this 'Node' yet
var latestGossip: Gossip = Gossip()
var joinInProgress: Map[Address, Deadline] = Map.empty
var stats = ClusterStats()
val heartbeatSender = context.actorOf(Props(new ClusterHeartbeatSender(environment)).
withDispatcher(UseDispatcher), name = "heartbeatSender")
val coreSender = context.actorOf(Props(new ClusterCoreSender(selfAddress)).
withDispatcher(UseDispatcher), name = "coreSender")
// start periodic gossip to random nodes in cluster
val gossipTask =
FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(GossipInterval), GossipInterval) {
self ! GossipTick
}
// start periodic heartbeat to all nodes in cluster
val heartbeatTask =
FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(HeartbeatInterval), HeartbeatInterval) {
self ! HeartbeatTick
}
// start periodic cluster failure detector reaping (moving nodes condemned by the failure detector to unreachable list)
val failureDetectorReaperTask =
FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(UnreachableNodesReaperInterval), UnreachableNodesReaperInterval) {
self ! ReapUnreachableTick
}
// start periodic leader action management (only applies for the current leader)
private val leaderActionsTask =
FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(LeaderActionsInterval), LeaderActionsInterval) {
self ! LeaderActionsTick
}
// start periodic publish of current state
private val publishStateTask: Option[Cancellable] =
if (PublishStateInterval == Duration.Zero) None
else Some(FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(PublishStateInterval), PublishStateInterval) {
self ! PublishStateTick
})
override def preStart(): Unit = {
if (AutoJoin) self ! InternalClusterAction.JoinSeedNode
}
override def postStop(): Unit = {
gossipTask.cancel()
heartbeatTask.cancel()
failureDetectorReaperTask.cancel()
leaderActionsTask.cancel()
publishStateTask foreach { _.cancel() }
}
def receive = {
case msg: GossipEnvelope receiveGossip(msg)
case msg: GossipMergeConflict receiveGossipMerge(msg)
case GossipTick gossip()
case HeartbeatTick heartbeat()
case ReapUnreachableTick reapUnreachableMembers()
case LeaderActionsTick leaderActions()
case PublishStateTick publishState()
case JoinSeedNode joinSeedNode()
case InitJoin initJoin()
case InitJoinAck(address) join(address)
case Failure(e: AskTimeoutException) joinSeedNodeTimeout()
case JoinTo(address) join(address)
case ClusterUserAction.Join(address) joining(address)
case ClusterUserAction.Down(address) downing(address)
case ClusterUserAction.Leave(address) leaving(address)
case Exit(address) exiting(address)
case Remove(address) removing(address)
case SendGossipTo(address) gossipTo(address)
case p: Ping ping(p)
}
def joinSeedNode(): Unit = {
val seedRoutees = environment.seedNodes.collect { case a if a != selfAddress self.path.toStringWithAddress(a) }
if (seedRoutees.isEmpty) {
join(selfAddress)
} else {
implicit val within = Timeout(SeedNodeTimeout)
val seedRouter = context.actorOf(
Props.empty.withRouter(ScatterGatherFirstCompletedRouter(
routees = seedRoutees, within = within.duration)))
seedRouter ? InitJoin pipeTo self
seedRouter ! PoisonPill
}
}
def initJoin(): Unit = sender ! InitJoinAck(selfAddress)
def joinSeedNodeTimeout(): Unit = join(selfAddress)
/**
* Try to join this cluster node with the node specified by 'address'.
* A 'Join(thisNodeAddress)' command is sent to the node to join.
*/
def join(address: Address): Unit = {
val localGossip = latestGossip
// wipe our state since a node that joins a cluster must be empty
latestGossip = Gossip()
joinInProgress = Map(address -> (Deadline.now + JoinTimeout))
// wipe the failure detector since we are starting fresh and shouldn't care about the past
failureDetector.reset()
notifyListeners(localGossip)
val command = ClusterUserAction.Join(selfAddress)
coreSender ! SendClusterMessage(address, command)
}
/**
* State transition to JOINING - new node joining.
*/
def joining(node: Address): Unit = {
val localGossip = latestGossip
val localMembers = localGossip.members
val localUnreachable = localGossip.overview.unreachable
val alreadyMember = localMembers.exists(_.address == node)
val isUnreachable = localGossip.overview.isNonDownUnreachable(node)
if (!alreadyMember && !isUnreachable) {
// remove the node from the 'unreachable' set in case it is a DOWN node that is rejoining cluster
val (rejoiningMember, newUnreachableMembers) = localUnreachable partition { _.address == node }
val newOverview = localGossip.overview copy (unreachable = newUnreachableMembers)
// remove the node from the failure detector if it is a DOWN node that is rejoining cluster
if (rejoiningMember.nonEmpty) failureDetector.remove(node)
// add joining node as Joining
// add self in case someone else joins before self has joined (Set discards duplicates)
val newMembers = localMembers :+ Member(node, Joining) :+ Member(selfAddress, Joining)
val newGossip = localGossip copy (overview = newOverview, members = newMembers)
val versionedGossip = newGossip :+ vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
latestGossip = seenVersionedGossip
log.debug("Cluster Node [{}] - Node [{}] is JOINING", selfAddress, node)
// treat join as initial heartbeat, so that it becomes unavailable if nothing more happens
if (node != selfAddress) {
failureDetector heartbeat node
gossipTo(node)
}
notifyListeners(localGossip)
}
}
/**
* State transition to LEAVING.
*/
def leaving(address: Address): Unit = {
val localGossip = latestGossip
if (localGossip.members.exists(_.address == address)) { // only try to update if the node is available (in the member ring)
val newMembers = localGossip.members map { member if (member.address == address) Member(address, Leaving) else member } // mark node as LEAVING
val newGossip = localGossip copy (members = newMembers)
val versionedGossip = newGossip :+ vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
latestGossip = seenVersionedGossip
log.info("Cluster Node [{}] - Marked address [{}] as LEAVING", selfAddress, address)
notifyListeners(localGossip)
}
}
/**
* State transition to EXITING.
*/
def exiting(address: Address): Unit = {
log.info("Cluster Node [{}] - Marked node [{}] as EXITING", selfAddress, address)
// FIXME implement when we implement hand-off
}
/**
* State transition to REMOVED.
*
* This method is for now only called after the LEADER have sent a Removed message - telling the node
* to shut down himself.
*
* In the future we might change this to allow the USER to send a Removed(address) message telling an
* arbitrary node to be moved direcly from UP -> REMOVED.
*/
def removing(address: Address): Unit = {
log.info("Cluster Node [{}] - Node has been REMOVED by the leader - shutting down...", selfAddress)
val localGossip = latestGossip
// just cleaning up the gossip state
latestGossip = Gossip()
// make sure the final (removed) state is always published
notifyListeners(localGossip)
environment.shutdown()
}
/**
* The node to DOWN is removed from the 'members' set and put in the 'unreachable' set (if not already there)
* and its status is set to DOWN. The node is also removed from the 'seen' table.
*
* The node will reside as DOWN in the 'unreachable' set until an explicit command JOIN command is sent directly
* to this node and it will then go through the normal JOINING procedure.
*/
def downing(address: Address): Unit = {
val localGossip = latestGossip
val localMembers = localGossip.members
val localOverview = localGossip.overview
val localSeen = localOverview.seen
val localUnreachableMembers = localOverview.unreachable
// 1. check if the node to DOWN is in the 'members' set
val downedMember: Option[Member] = localMembers.collectFirst {
case m if m.address == address m.copy(status = Down)
}
val newMembers = downedMember match {
case Some(m)
log.info("Cluster Node [{}] - Marking node [{}] as DOWN", selfAddress, m.address)
localMembers - m
case None localMembers
}
// 2. check if the node to DOWN is in the 'unreachable' set
val newUnreachableMembers =
localUnreachableMembers.map { member
// no need to DOWN members already DOWN
if (member.address == address && member.status != Down) {
log.info("Cluster Node [{}] - Marking unreachable node [{}] as DOWN", selfAddress, member.address)
member copy (status = Down)
} else member
}
// 3. add the newly DOWNED members from the 'members' (in step 1.) to the 'newUnreachableMembers' set.
val newUnreachablePlusNewlyDownedMembers = newUnreachableMembers ++ downedMember
// 4. remove nodes marked as DOWN from the 'seen' table
val newSeen = localSeen -- newUnreachablePlusNewlyDownedMembers.collect {
case m if m.status == Down m.address
}
// update gossip overview
val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachablePlusNewlyDownedMembers)
val newGossip = localGossip copy (overview = newOverview, members = newMembers) // update gossip
val versionedGossip = newGossip :+ vclockNode
latestGossip = versionedGossip seen selfAddress
notifyListeners(localGossip)
}
/**
* When conflicting versions of received and local [[akka.cluster.Gossip]] is detected
* it's forwarded to the leader for conflict resolution. Trying to simultaneously
* resolving conflicts at several nodes creates new conflicts. Therefore the leader resolves
* conflicts to limit divergence. To avoid overload there is also a configurable rate
* limit of how many conflicts that are handled by second. If the limit is
* exceeded the conflicting gossip messages are dropped and will reappear later.
*/
def receiveGossipMerge(merge: GossipMergeConflict): Unit = {
stats = stats.incrementMergeConflictCount
val rate = mergeRate(stats.mergeConflictCount)
if (rate <= MaxGossipMergeRate) {
receiveGossip(merge.a.copy(conversation = false))
receiveGossip(merge.b.copy(conversation = false))
// use one-way gossip from leader to reduce load of leader
def sendBack(to: Address): Unit = {
if (to != selfAddress && !latestGossip.overview.unreachable.exists(_.address == to))
oneWayGossipTo(to)
}
sendBack(merge.a.from)
sendBack(merge.b.from)
} else {
log.debug("Dropping gossip merge conflict due to rate [{}] / s ", rate)
}
}
/**
* Receive new gossip.
*/
def receiveGossip(envelope: GossipEnvelope): Unit = {
val from = envelope.from
val remoteGossip = envelope.gossip
val localGossip = latestGossip
if (remoteGossip.overview.unreachable.exists(_.address == selfAddress)) {
// FIXME how should we handle this situation?
log.debug("Received gossip with self as unreachable, from [{}]", from)
} else if (!localGossip.overview.isNonDownUnreachable(from)) {
// leader handles merge conflicts, or when they have different views of how is leader
val handleMerge = localGossip.leader == Some(selfAddress) || localGossip.leader != remoteGossip.leader
val conflict = remoteGossip.version <> localGossip.version
if (conflict && !handleMerge) {
// delegate merge resolution to leader to reduce number of simultaneous resolves,
// which will result in new conflicts
stats = stats.incrementMergeDetectedCount
log.debug("Merge conflict [{}] detected [{}] <> [{}]", stats.mergeDetectedCount, selfAddress, from)
stats = stats.incrementMergeConflictCount
val rate = mergeRate(stats.mergeConflictCount)
if (rate <= MaxGossipMergeRate) {
coreSender ! SendClusterMessage(
to = localGossip.leader.get,
msg = GossipMergeConflict(GossipEnvelope(selfAddress, localGossip), envelope))
} else {
log.debug("Skipping gossip merge conflict due to rate [{}] / s ", rate)
}
} else {
val winningGossip =
if (conflict) {
// conflicting versions, merge, and new version
val mergedGossip = remoteGossip merge localGossip
mergedGossip :+ vclockNode
} else if (remoteGossip.version < localGossip.version) {
// local gossip is newer
localGossip
} else {
// remote gossip is newer
remoteGossip
}
val newJoinInProgress =
if (joinInProgress.isEmpty) joinInProgress
else joinInProgress --
winningGossip.members.map(_.address) --
winningGossip.overview.unreachable.map(_.address)
latestGossip = winningGossip seen selfAddress
joinInProgress = newJoinInProgress
// for all new joining nodes we remove them from the failure detector
(latestGossip.members -- localGossip.members).filter(_.status == Joining).foreach { node
failureDetector.remove(node.address)
}
log.debug("Cluster Node [{}] - Receiving gossip from [{}]", selfAddress, from)
if (conflict) {
stats = stats.incrementMergeCount
log.debug(
"""Couldn't establish a causal relationship between "remote" gossip and "local" gossip - Remote[{}] - Local[{}] - merged them into [{}]""",
remoteGossip, localGossip, winningGossip)
}
stats = stats.incrementReceivedGossipCount
notifyListeners(localGossip)
if (envelope.conversation &&
(conflict || (winningGossip ne remoteGossip) || (latestGossip ne remoteGossip))) {
// send back gossip to sender when sender had different view, i.e. merge, or sender had
// older or sender had newer
gossipTo(from)
}
}
}
}
def mergeRate(count: Long): Double = (count * 1000.0) / GossipInterval.toMillis
/**
* Initiates a new round of gossip.
*/
def gossip(): Unit = {
stats = stats.copy(mergeConflictCount = 0)
log.debug("Cluster Node [{}] - Initiating new round of gossip", selfAddress)
if (!isSingletonCluster && isAvailable) {
val localGossip = latestGossip
// important to not accidentally use `map` of the SortedSet, since the original order is not preserved
val localMembers = localGossip.members.toIndexedSeq
val localMembersSize = localMembers.size
val localMemberAddresses = localMembers map { _.address }
val localUnreachableMembers = localGossip.overview.unreachable.toIndexedSeq
val localUnreachableSize = localUnreachableMembers.size
// gossip to a random alive member with preference to a member
// with older or newer gossip version
val nodesWithdifferentView = {
val localMemberAddressesSet = localGossip.members map { _.address }
for {
(address, version) localGossip.overview.seen
if localMemberAddressesSet contains address
if version != localGossip.version
} yield address
}
val gossipedToAlive =
if (nodesWithdifferentView.nonEmpty && ThreadLocalRandom.current.nextDouble() < GossipDifferentViewProbability)
gossipToRandomNodeOf(nodesWithdifferentView.toIndexedSeq)
else
gossipToRandomNodeOf(localMemberAddresses)
}
}
/**
* Runs periodic leader actions, such as auto-downing unreachable nodes, assigning partitions etc.
*/
def leaderActions(): Unit = {
val localGossip = latestGossip
val localMembers = localGossip.members
val isLeader = localMembers.nonEmpty && (selfAddress == localMembers.head.address)
if (isLeader && isAvailable) {
// only run the leader actions if we are the LEADER and available
val localOverview = localGossip.overview
val localSeen = localOverview.seen
val localUnreachableMembers = localOverview.unreachable
val hasPartionHandoffCompletedSuccessfully: Boolean = {
// FIXME implement partion handoff and a check if it is completed - now just returns TRUE - e.g. has completed successfully
true
}
// Leader actions are as follows:
// 1. Move EXITING => REMOVED -- When all nodes have seen that the node is EXITING (convergence) - remove the nodes from the node ring and seen table
// 2. Move JOINING => UP -- When a node joins the cluster
// 3. Move LEAVING => EXITING -- When all partition handoff has completed
// 4. Move UNREACHABLE => DOWN -- When the node is in the UNREACHABLE set it can be auto-down by leader
// 5. Store away all stuff needed for the side-effecting processing in 10.
// 6. Updating the vclock version for the changes
// 7. Updating the 'seen' table
// 8. Try to update the state with the new gossip
// 9. If failure - retry
// 10. If success - run all the side-effecting processing
val (
newGossip: Gossip,
hasChangedState: Boolean,
upMembers,
exitingMembers,
removedMembers,
unreachableButNotDownedMembers) =
if (localGossip.convergence) {
// we have convergence - so we can't have unreachable nodes
// transform the node member ring - filterNot/map/map
val newMembers =
localMembers filterNot { member
// ----------------------
// 1. Move EXITING => REMOVED - e.g. remove the nodes from the 'members' set/node ring and seen table
// ----------------------
member.status == MemberStatus.Exiting
} map { member
// ----------------------
// 2. Move JOINING => UP (once all nodes have seen that this node is JOINING e.g. we have a convergence)
// ----------------------
if (member.status == Joining) member copy (status = Up)
else member
} map { member
// ----------------------
// 3. Move LEAVING => EXITING (once we have a convergence on LEAVING *and* if we have a successful partition handoff)
// ----------------------
if (member.status == Leaving && hasPartionHandoffCompletedSuccessfully) member copy (status = Exiting)
else member
}
// ----------------------
// 5. Store away all stuff needed for the side-effecting processing in 10.
// ----------------------
// Check for the need to do side-effecting on successful state change
// Repeat the checking for transitions between JOINING -> UP, LEAVING -> EXITING, EXITING -> REMOVED
// to check for state-changes and to store away removed and exiting members for later notification
// 1. check for state-changes to update
// 2. store away removed and exiting members so we can separate the pure state changes (that can be retried on collision) and the side-effecting message sending
val (removedMembers, newMembers1) = localMembers partition (_.status == Exiting)
val (upMembers, newMembers2) = newMembers1 partition (_.status == Joining)
val (exitingMembers, newMembers3) = newMembers2 partition (_.status == Leaving && hasPartionHandoffCompletedSuccessfully)
val hasChangedState = removedMembers.nonEmpty || upMembers.nonEmpty || exitingMembers.nonEmpty
// removing REMOVED nodes from the 'seen' table
val newSeen = localSeen -- removedMembers.map(_.address)
// removing REMOVED nodes from the 'unreachable' set
val newUnreachableMembers = localUnreachableMembers -- removedMembers
val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachableMembers) // update gossip overview
val newGossip = localGossip copy (members = newMembers, overview = newOverview) // update gossip
(newGossip, hasChangedState, upMembers, exitingMembers, removedMembers, Set.empty[Member])
} else if (AutoDown) {
// we don't have convergence - so we might have unreachable nodes
// if 'auto-down' is turned on, then try to auto-down any unreachable nodes
val newUnreachableMembers = localUnreachableMembers.map { member
// ----------------------
// 5. Move UNREACHABLE => DOWN (auto-downing by leader)
// ----------------------
if (member.status == Down) member // no need to DOWN members already DOWN
else member copy (status = Down)
}
// Check for the need to do side-effecting on successful state change
val (unreachableButNotDownedMembers, _) = localUnreachableMembers partition (_.status != Down)
// removing nodes marked as DOWN from the 'seen' table
val newSeen = localSeen -- newUnreachableMembers.collect { case m if m.status == Down m.address }
val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachableMembers) // update gossip overview
val newGossip = localGossip copy (overview = newOverview) // update gossip
(newGossip, unreachableButNotDownedMembers.nonEmpty, Set.empty[Member], Set.empty[Member], Set.empty[Member], unreachableButNotDownedMembers)
} else (localGossip, false, Set.empty[Member], Set.empty[Member], Set.empty[Member], Set.empty[Member])
if (hasChangedState) { // we have a change of state - version it and try to update
// ----------------------
// 6. Updating the vclock version for the changes
// ----------------------
val versionedGossip = newGossip :+ vclockNode
// ----------------------
// 7. Updating the 'seen' table
// Unless the leader (this node) is part of the removed members, i.e. the leader have moved himself from EXITING -> REMOVED
// ----------------------
val seenVersionedGossip =
if (removedMembers.exists(_.address == selfAddress)) versionedGossip
else versionedGossip seen selfAddress
// ----------------------
// 8. Update the state with the new gossip
// ----------------------
latestGossip = seenVersionedGossip
// ----------------------
// 9. Run all the side-effecting processing
// ----------------------
// log the move of members from joining to up
upMembers foreach { member log.info("Cluster Node [{}] - Leader is moving node [{}] from JOINING to UP", selfAddress, member.address) }
// tell all removed members to remove and shut down themselves
removedMembers foreach { member
val address = member.address
log.info("Cluster Node [{}] - Leader is moving node [{}] from EXITING to REMOVED - and removing node from node ring", selfAddress, address)
coreSender ! SendClusterMessage(
to = address,
msg = ClusterLeaderAction.Remove(address))
}
// tell all exiting members to exit
exitingMembers foreach { member
val address = member.address
log.info("Cluster Node [{}] - Leader is moving node [{}] from LEAVING to EXITING", selfAddress, address)
coreSender ! SendClusterMessage(
to = address,
msg = ClusterLeaderAction.Exit(address)) // FIXME should use ? to await completion of handoff?
}
// log the auto-downing of the unreachable nodes
unreachableButNotDownedMembers foreach { member
log.info("Cluster Node [{}] - Leader is marking unreachable node [{}] as DOWN", selfAddress, member.address)
}
notifyListeners(localGossip)
}
}
}
def heartbeat(): Unit = {
removeOverdueJoinInProgress()
val beatTo = latestGossip.members.toSeq.map(_.address) ++ joinInProgress.keys
val deadline = Deadline.now + HeartbeatInterval
for (address beatTo; if address != selfAddress)
heartbeatSender ! SendHeartbeat(selfHeartbeat, address, deadline)
}
/**
* Removes overdue joinInProgress from State.
*/
def removeOverdueJoinInProgress(): Unit = {
val overdueJoins = joinInProgress collect {
case (address, deadline) if deadline.isOverdue address
}
if (overdueJoins.nonEmpty) {
joinInProgress = joinInProgress -- overdueJoins
}
}
/**
* Reaps the unreachable members (moves them to the 'unreachable' list in the cluster overview) according to the failure detector's verdict.
*/
def reapUnreachableMembers(): Unit = {
if (!isSingletonCluster && isAvailable) {
// only scrutinize if we are a non-singleton cluster and available
val localGossip = latestGossip
val localOverview = localGossip.overview
val localMembers = localGossip.members
val localUnreachableMembers = localGossip.overview.unreachable
val newlyDetectedUnreachableMembers = localMembers filterNot { member
member.address == selfAddress || failureDetector.isAvailable(member.address)
}
if (newlyDetectedUnreachableMembers.nonEmpty) {
val newMembers = localMembers -- newlyDetectedUnreachableMembers
val newUnreachableMembers = localUnreachableMembers ++ newlyDetectedUnreachableMembers
val newOverview = localOverview copy (unreachable = newUnreachableMembers)
val newGossip = localGossip copy (overview = newOverview, members = newMembers)
// updating vclock and 'seen' table
val versionedGossip = newGossip :+ vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
latestGossip = seenVersionedGossip
log.error("Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]", selfAddress, newlyDetectedUnreachableMembers.mkString(", "))
notifyListeners(localGossip)
}
}
}
def seedNodes: IndexedSeq[Address] = environment.seedNodes
def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] =
if (addresses.isEmpty) None
else Some(addresses(ThreadLocalRandom.current nextInt addresses.size))
def isSingletonCluster: Boolean = latestGossip.isSingletonCluster
def isAvailable: Boolean = latestGossip.isAvailable(selfAddress)
/**
* Gossips latest gossip to a random member in the set of members passed in as argument.
*
* @return the used [[akka.actor.Address] if any
*/
private def gossipToRandomNodeOf(addresses: IndexedSeq[Address]): Option[Address] = {
log.debug("Cluster Node [{}] - Selecting random node to gossip to [{}]", selfAddress, addresses.mkString(", "))
// filter out myself
val peer = selectRandomNode(addresses filterNot (_ == selfAddress))
peer foreach gossipTo
peer
}
/**
* Gossips latest gossip to an address.
*/
def gossipTo(address: Address): Unit =
gossipTo(address, GossipEnvelope(selfAddress, latestGossip, conversation = true))
def oneWayGossipTo(address: Address): Unit =
gossipTo(address, GossipEnvelope(selfAddress, latestGossip, conversation = false))
def gossipTo(address: Address, gossipMsg: GossipEnvelope): Unit = if (address != selfAddress)
coreSender ! SendClusterMessage(address, gossipMsg)
def notifyListeners(oldGossip: Gossip): Unit = {
if (PublishStateInterval == Duration.Zero) publishState()
val oldMembersStatus = oldGossip.members.map(m (m.address, m.status))
val newMembersStatus = latestGossip.members.map(m (m.address, m.status))
if (newMembersStatus != oldMembersStatus)
environment notifyMembershipChangeListeners latestGossip.members
}
def publishState(): Unit = {
environment.publishLatestGossip(latestGossip)
environment.publishLatestStats(stats)
}
def ping(p: Ping): Unit = sender ! Pong(p)
}
/**
* INTERNAL API.
*/
private[cluster] final class ClusterCoreSender(selfAddress: Address) extends Actor with ActorLogging {
import InternalClusterAction._
/**
* Looks up and returns the remote cluster command connection for the specific address.
*/
private def clusterCoreConnectionFor(address: Address): ActorRef =
context.system.actorFor(RootActorPath(address) / "system" / "cluster" / "core")
def receive = {
case SendClusterMessage(to, msg)
log.debug("Cluster Node [{}] - Trying to send [{}] to [{}]", selfAddress, msg.getClass.getSimpleName, to)
clusterCoreConnectionFor(to) ! msg
}
}
/**
* INTERNAL API
*/
private[cluster] case class ClusterStats(
receivedGossipCount: Long = 0L,
mergeConflictCount: Long = 0L,
mergeCount: Long = 0L,
mergeDetectedCount: Long = 0L) {
def incrementReceivedGossipCount(): ClusterStats =
copy(receivedGossipCount = receivedGossipCount + 1)
def incrementMergeConflictCount(): ClusterStats =
copy(mergeConflictCount = mergeConflictCount + 1)
def incrementMergeCount(): ClusterStats =
copy(mergeCount = mergeCount + 1)
def incrementMergeDetectedCount(): ClusterStats =
copy(mergeDetectedCount = mergeDetectedCount + 1)
}

View file

@ -0,0 +1,128 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import language.postfixOps
import akka.actor.{ ReceiveTimeout, ActorLogging, ActorRef, Address, Actor, RootActorPath, Props }
import java.security.MessageDigest
import akka.pattern.{ CircuitBreaker, CircuitBreakerOpenException }
import scala.concurrent.util.duration._
import scala.concurrent.util.Deadline
/**
* Sent at regular intervals for failure detection.
*/
case class Heartbeat(from: Address) extends ClusterMessage
/**
* INTERNAL API.
*
* Receives Heartbeat messages and delegates to Cluster.
* Instantiated as a single instance for each Cluster - e.g. heartbeats are serialized
* to Cluster message after message, but concurrent with other types of messages.
*/
private[cluster] final class ClusterHeartbeatDaemon(environment: ClusterEnvironment) extends Actor with ActorLogging {
def receive = {
case Heartbeat(from) environment.failureDetector heartbeat from
}
}
/**
* INTERNAL API
*/
private[cluster] object ClusterHeartbeatSender {
/**
*
* Command to [akka.cluster.ClusterHeartbeatSender]], which will send [[akka.cluster.Heartbeat]]
* to the other node.
* Local only, no need to serialize.
*/
case class SendHeartbeat(heartbeatMsg: Heartbeat, to: Address, deadline: Deadline)
}
/*
* INTERNAL API
*
* This actor is responsible for sending the heartbeat messages to
* other nodes. Netty blocks when sending to broken connections. This actor
* isolates sending to different nodes by using child workers for each target
* address and thereby reduce the risk of irregular heartbeats to healty
* nodes due to broken connections to other nodes.
*/
private[cluster] final class ClusterHeartbeatSender(environment: ClusterEnvironment) extends Actor with ActorLogging {
import ClusterHeartbeatSender._
/**
* Looks up and returns the remote cluster heartbeat connection for the specific address.
*/
def clusterHeartbeatConnectionFor(address: Address): ActorRef =
context.system.actorFor(RootActorPath(address) / "system" / "cluster" / "heartbeat")
val digester = MessageDigest.getInstance("MD5")
/**
* Child name is MD5 hash of the address.
* FIXME Change to URLEncode when ticket #2123 has been fixed
*/
def encodeChildName(name: String): String = {
digester update name.getBytes("UTF-8")
digester.digest.map { h "%02x".format(0xFF & h) }.mkString
}
def receive = {
case msg @ SendHeartbeat(from, to, deadline)
val workerName = encodeChildName(to.toString)
val worker = context.actorFor(workerName) match {
case notFound if notFound.isTerminated
context.actorOf(Props(new ClusterHeartbeatSenderWorker(
environment.settings.SendCircuitBreakerSettings, clusterHeartbeatConnectionFor(to))), workerName)
case child child
}
worker ! msg
}
}
/**
* Responsible for sending [[akka.cluster.Heartbeat]] to one specific address.
*
* Netty blocks when sending to broken connections, and this actor uses
* a configurable circuit breaker to reduce connect attempts to broken
* connections.
*
* @see ClusterHeartbeatSender
*/
private[cluster] final class ClusterHeartbeatSenderWorker(
cbSettings: CircuitBreakerSettings, toRef: ActorRef)
extends Actor with ActorLogging {
import ClusterHeartbeatSender._
val breaker = CircuitBreaker(context.system.scheduler,
cbSettings.maxFailures, cbSettings.callTimeout, cbSettings.resetTimeout).
onHalfOpen(log.debug("CircuitBreaker Half-Open for: [{}]", toRef)).
onOpen(log.debug("CircuitBreaker Open for [{}]", toRef)).
onClose(log.debug("CircuitBreaker Closed for [{}]", toRef))
// make sure it will cleanup when not used any more
context.setReceiveTimeout(30 seconds)
def receive = {
case SendHeartbeat(heartbeatMsg, _, deadline)
if (!deadline.isOverdue) {
// the CircuitBreaker will measure elapsed time and open if too many long calls
try breaker.withSyncCircuitBreaker {
log.debug("Cluster Node [{}] - Heartbeat to [{}]", heartbeatMsg.from, toRef)
toRef ! heartbeatMsg
if (deadline.isOverdue) log.debug("Sending heartbeat to [{}] took longer than expected", toRef)
} catch { case e: CircuitBreakerOpenException /* skip sending heartbeat to broken connection */ }
}
case ReceiveTimeout context.stop(self) // cleanup when not used
}
}

View file

@ -0,0 +1,107 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import java.lang.management.ManagementFactory
import javax.management.StandardMBean
import akka.event.LoggingAdapter
import akka.actor.AddressFromURIString
import javax.management.ObjectName
import javax.management.InstanceAlreadyExistsException
import javax.management.InstanceNotFoundException
/**
* Interface for the cluster JMX MBean.
*/
trait ClusterNodeMBean {
def getMemberStatus: String
def getClusterStatus: String
def getLeader: String
def isSingleton: Boolean
def isConvergence: Boolean
def isAvailable: Boolean
def isRunning: Boolean
def join(address: String)
def leave(address: String)
def down(address: String)
}
/**
* Internal API
*/
private[akka] class ClusterJmx(clusterNode: Cluster, log: LoggingAdapter) {
private val mBeanServer = ManagementFactory.getPlatformMBeanServer
private val clusterMBeanName = new ObjectName("akka:type=Cluster")
/**
* Creates the cluster JMX MBean and registers it in the MBean server.
*/
def createMBean() = {
val mbean = new StandardMBean(classOf[ClusterNodeMBean]) with ClusterNodeMBean {
// JMX attributes (bean-style)
/*
* Sends a string to the JMX client that will list all nodes in the node ring as follows:
* {{{
* Members:
* Member(address = akka://system0@localhost:5550, status = Up)
* Member(address = akka://system1@localhost:5551, status = Up)
* Unreachable:
* Member(address = akka://system2@localhost:5553, status = Down)
* }}}
*/
def getClusterStatus: String = {
val gossip = clusterNode.latestGossip
val unreachable = gossip.overview.unreachable
val metaData = gossip.meta
"\nMembers:\n\t" + gossip.members.mkString("\n\t") +
{ if (unreachable.nonEmpty) "\nUnreachable:\n\t" + unreachable.mkString("\n\t") else "" } +
{ if (metaData.nonEmpty) "\nMeta Data:\t" + metaData.toString else "" }
}
def getMemberStatus: String = clusterNode.status.toString
def getLeader: String = clusterNode.leader.toString
def isSingleton: Boolean = clusterNode.isSingletonCluster
def isConvergence: Boolean = clusterNode.convergence.isDefined
def isAvailable: Boolean = clusterNode.isAvailable
def isRunning: Boolean = clusterNode.isRunning
// JMX commands
def join(address: String) = clusterNode.join(AddressFromURIString(address))
def leave(address: String) = clusterNode.leave(AddressFromURIString(address))
def down(address: String) = clusterNode.down(AddressFromURIString(address))
}
try {
mBeanServer.registerMBean(mbean, clusterMBeanName)
log.info("Cluster Node [{}] - registered cluster JMX MBean [{}]", clusterNode.selfAddress, clusterMBeanName)
} catch {
case e: InstanceAlreadyExistsException // ignore - we are running multiple cluster nodes in the same JVM (probably for testing)
}
}
/**
* Unregisters the cluster JMX MBean from MBean server.
*/
def unregisterMBean(): Unit = {
try {
mBeanServer.unregisterMBean(clusterMBeanName)
} catch {
case e: InstanceNotFoundException // ignore - we are running multiple cluster nodes in the same JVM (probably for testing)
}
}
}

View file

@ -10,6 +10,7 @@ import akka.ConfigurationException
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import akka.actor.Address import akka.actor.Address
import akka.actor.AddressFromURIString import akka.actor.AddressFromURIString
import akka.dispatch.Dispatchers
class ClusterSettings(val config: Config, val systemName: String) { class ClusterSettings(val config: Config, val systemName: String) {
import config._ import config._
@ -31,12 +32,22 @@ class ClusterSettings(val config: Config, val systemName: String) {
final val HeartbeatInterval: Duration = Duration(getMilliseconds("akka.cluster.heartbeat-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 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 UnreachableNodesReaperInterval: Duration = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS)
final val NrOfGossipDaemons: Int = getInt("akka.cluster.nr-of-gossip-daemons") final val PublishStateInterval: Duration = Duration(getMilliseconds("akka.cluster.publish-state-interval"), MILLISECONDS)
final val NrOfDeputyNodes: Int = getInt("akka.cluster.nr-of-deputy-nodes")
final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join") final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join")
final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down") final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down")
final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS) final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS)
final val UseDispatcher: String = getString("akka.cluster.use-dispatcher") match {
case "" Dispatchers.DefaultDispatcherId
case id id
}
final val GossipDifferentViewProbability: Double = getDouble("akka.cluster.gossip-different-view-probability") final val GossipDifferentViewProbability: Double = getDouble("akka.cluster.gossip-different-view-probability")
final val MaxGossipMergeRate: Double = getDouble("akka.cluster.max-gossip-merge-rate")
final val SchedulerTickDuration: Duration = Duration(getMilliseconds("akka.cluster.scheduler.tick-duration"), MILLISECONDS) final val SchedulerTickDuration: Duration = Duration(getMilliseconds("akka.cluster.scheduler.tick-duration"), MILLISECONDS)
final val SchedulerTicksPerWheel: Int = getInt("akka.cluster.scheduler.ticks-per-wheel") final val SchedulerTicksPerWheel: Int = getInt("akka.cluster.scheduler.ticks-per-wheel")
final val SendCircuitBreakerSettings: CircuitBreakerSettings = CircuitBreakerSettings(
maxFailures = getInt("akka.cluster.send-circuit-breaker.max-failures"),
callTimeout = Duration(getMilliseconds("akka.cluster.send-circuit-breaker.call-timeout"), MILLISECONDS),
resetTimeout = Duration(getMilliseconds("akka.cluster.send-circuit-breaker.reset-timeout"), MILLISECONDS))
} }
case class CircuitBreakerSettings(maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)

View file

@ -5,19 +5,16 @@
package akka.cluster package akka.cluster
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLong }
import java.util.concurrent.atomic.AtomicLong import akka.actor.{ Scheduler, Cancellable }
import akka.actor.Scheduler
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object FixedRateTask { private[akka] object FixedRateTask {
def apply(scheduler: Scheduler, initalDelay: Duration, delay: Duration)(f: Unit): FixedRateTask = { def apply(scheduler: Scheduler, initalDelay: Duration, delay: Duration)(f: Unit): FixedRateTask =
new FixedRateTask(scheduler, initalDelay, delay, new Runnable { def run(): Unit = f }) new FixedRateTask(scheduler, initalDelay, delay, new Runnable { def run(): Unit = f })
}
} }
/** /**
@ -27,7 +24,8 @@ private[akka] object FixedRateTask {
* for inaccuracy in scheduler. It will start when constructed, using the * for inaccuracy in scheduler. It will start when constructed, using the
* initialDelay. * initialDelay.
*/ */
private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, delay: Duration, task: Runnable) extends Runnable { private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, delay: Duration, task: Runnable)
extends Runnable with Cancellable {
private val delayNanos = delay.toNanos private val delayNanos = delay.toNanos
private val cancelled = new AtomicBoolean(false) private val cancelled = new AtomicBoolean(false)
@ -37,9 +35,11 @@ private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, d
def cancel(): Unit = cancelled.set(true) def cancel(): Unit = cancelled.set(true)
override final def run(): Unit = if (!cancelled.get) try { def isCancelled: Boolean = cancelled.get
override final def run(): Unit = if (!isCancelled) try {
task.run() task.run()
} finally if (!cancelled.get) { } finally if (!isCancelled) {
val nextTime = startTime + delayNanos * counter.incrementAndGet val nextTime = startTime + delayNanos * counter.incrementAndGet
// it's ok to schedule with negative duration, will run asap // it's ok to schedule with negative duration, will run asap
val nextDelay = Duration(nextTime - System.nanoTime, TimeUnit.NANOSECONDS) val nextDelay = Duration(nextTime - System.nanoTime, TimeUnit.NANOSECONDS)

View file

@ -0,0 +1,212 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor.Address
import scala.collection.immutable.SortedSet
import MemberStatus._
object Gossip {
val emptyMembers: SortedSet[Member] = SortedSet.empty
}
/**
* Represents the state of the cluster; cluster ring membership, ring convergence, meta data -
* all versioned by a vector clock.
*
* When a node is joining the `Member`, with status `Joining`, is added to `members`.
* If the joining node was downed it is moved from `overview.unreachable` (status `Down`)
* to `members` (status `Joining`). It cannot rejoin if not first downed.
*
* When convergence is reached the leader change status of `members` from `Joining`
* to `Up`.
*
* When failure detector consider a node as unavailable it will be moved from
* `members` to `overview.unreachable`.
*
* When a node is downed, either manually or automatically, its status is changed to `Down`.
* It is also removed from `overview.seen` table. The node will reside as `Down` in the
* `overview.unreachable` set until joining again and it will then go through the normal
* joining procedure.
*
* When a `Gossip` is received the version (vector clock) is used to determine if the
* received `Gossip` is newer or older than the current local `Gossip`. The received `Gossip`
* and local `Gossip` is merged in case of conflicting version, i.e. vector clocks without
* same history. When merged the seen table is cleared.
*
* When a node is told by the user to leave the cluster the leader will move it to `Leaving`
* and then rebalance and repartition the cluster and start hand-off by migrating the actors
* from the leaving node to the new partitions. Once this process is complete the leader will
* move the node to the `Exiting` state and once a convergence is complete move the node to
* `Removed` by removing it from the `members` set and sending a `Removed` command to the
* removed node telling it to shut itself down.
*/
case class Gossip(
overview: GossipOverview = GossipOverview(),
members: SortedSet[Member] = Gossip.emptyMembers, // sorted set of members with their status, sorted by address
meta: Map[String, Array[Byte]] = Map.empty,
version: VectorClock = VectorClock()) // vector clock version
extends ClusterMessage // is a serializable cluster message
with Versioned[Gossip] {
// FIXME can be disabled as optimization
assertInvariants
private def assertInvariants: Unit = {
val unreachableAndLive = members.intersect(overview.unreachable)
if (unreachableAndLive.nonEmpty)
throw new IllegalArgumentException("Same nodes in both members and unreachable is not allowed, got [%s]"
format unreachableAndLive.mkString(", "))
val allowedLiveMemberStatuses: Set[MemberStatus] = Set(Joining, Up, Leaving, Exiting)
def hasNotAllowedLiveMemberStatus(m: Member) = !allowedLiveMemberStatuses.contains(m.status)
if (members exists hasNotAllowedLiveMemberStatus)
throw new IllegalArgumentException("Live members must have status [%s], got [%s]"
format (allowedLiveMemberStatuses.mkString(", "),
(members filter hasNotAllowedLiveMemberStatus).mkString(", ")))
val seenButNotMember = overview.seen.keySet -- members.map(_.address) -- overview.unreachable.map(_.address)
if (seenButNotMember.nonEmpty)
throw new IllegalArgumentException("Nodes not part of cluster have marked the Gossip as seen, got [%s]"
format seenButNotMember.mkString(", "))
}
/**
* Increments the version for this 'Node'.
*/
def :+(node: VectorClock.Node): Gossip = copy(version = version :+ node)
/**
* Adds a member to the member node ring.
*/
def :+(member: Member): Gossip = {
if (members contains member) this
else this copy (members = members :+ member)
}
/**
* Marks the gossip as seen by this node (address) by updating the address entry in the 'gossip.overview.seen'
* Map with the VectorClock (version) for the new gossip.
*/
def seen(address: Address): Gossip = {
if (overview.seen.contains(address) && overview.seen(address) == version) this
else this copy (overview = overview copy (seen = overview.seen + (address -> version)))
}
/**
* Merges two Gossip instances including membership tables, meta-data tables and the VectorClock histories.
*/
def merge(that: Gossip): Gossip = {
import Member.ordering
// 1. merge vector clocks
val mergedVClock = this.version merge that.version
// 2. merge meta-data
val mergedMeta = this.meta ++ that.meta
// 3. merge unreachable by selecting the single Member with highest MemberStatus out of the Member groups
val mergedUnreachable = Member.pickHighestPriority(this.overview.unreachable, that.overview.unreachable)
// 4. merge members by selecting the single Member with highest MemberStatus out of the Member groups,
// and exclude unreachable
val mergedMembers = Gossip.emptyMembers :++ Member.pickHighestPriority(this.members, that.members).filterNot(mergedUnreachable.contains)
// 5. fresh seen table
val mergedSeen = Map.empty[Address, VectorClock]
Gossip(GossipOverview(mergedSeen, mergedUnreachable), mergedMembers, mergedMeta, mergedVClock)
}
/**
* Checks if we have a cluster convergence. If there are any unreachable nodes then we can't have a convergence -
* waiting for user to act (issuing DOWN) or leader to act (issuing DOWN through auto-down).
*
* @returns Some(convergedGossip) if convergence have been reached and None if not
*/
def convergence: Boolean = {
val unreachable = overview.unreachable
val seen = overview.seen
// First check that:
// 1. we don't have any members that are unreachable, or
// 2. all unreachable members in the set have status DOWN
// Else we can't continue to check for convergence
// When that is done we check that all the entries in the 'seen' table have the same vector clock version
// and that all members exists in seen table
val hasUnreachable = unreachable.nonEmpty && unreachable.exists { _.status != Down }
def allMembersInSeen = members.forall(m seen.contains(m.address))
def seenSame: Boolean =
if (seen.isEmpty) false
else {
val values = seen.values
val seenHead = values.head
values.forall(_ == seenHead)
}
!hasUnreachable && allMembersInSeen && seenSame
}
def isLeader(address: Address): Boolean =
members.nonEmpty && (address == members.head.address)
def leader: Option[Address] = members.headOption.map(_.address)
def isSingletonCluster: Boolean = members.size == 1
/**
* Returns true if the node is UP or JOINING.
*/
def isAvailable(address: Address): Boolean = !isUnavailable(address)
def isUnavailable(address: Address): Boolean = {
val isUnreachable = overview.unreachable exists { _.address == address }
val hasUnavailableMemberStatus = members exists { m m.status.isUnavailable && m.address == address }
isUnreachable || hasUnavailableMemberStatus
}
def member(address: Address): Member = {
members.find(_.address == address).orElse(overview.unreachable.find(_.address == address)).
getOrElse(Member(address, Removed))
}
override def toString =
"Gossip(" +
"overview = " + overview +
", members = [" + members.mkString(", ") +
"], meta = [" + meta.mkString(", ") +
"], version = " + version +
")"
}
/**
* Represents the overview of the cluster, holds the cluster convergence table and set with unreachable nodes.
*/
case class GossipOverview(
seen: Map[Address, VectorClock] = Map.empty,
unreachable: Set[Member] = Set.empty) {
def isNonDownUnreachable(address: Address): Boolean =
unreachable.exists { m m.address == address && m.status != Down }
override def toString =
"GossipOverview(seen = [" + seen.mkString(", ") +
"], unreachable = [" + unreachable.mkString(", ") +
"])"
}
/**
* Envelope adding a sender address to the gossip.
*/
case class GossipEnvelope(from: Address, gossip: Gossip, conversation: Boolean = true) extends ClusterMessage
/**
* When conflicting versions of received and local [[akka.cluster.Gossip]] is detected
* it's forwarded to the leader for conflict resolution.
*/
case class GossipMergeConflict(a: GossipEnvelope, b: GossipEnvelope) extends ClusterMessage

View file

@ -0,0 +1,117 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import scala.collection.GenTraversableOnce
import akka.actor.Address
import MemberStatus._
/**
* Represents the address and the current status of a cluster member node.
*
* Note: `hashCode` and `equals` are solely based on the underlying `Address`, not its `MemberStatus`.
*/
class Member(val address: Address, val status: MemberStatus) extends ClusterMessage {
override def hashCode = address.##
override def equals(other: Any) = Member.unapply(this) == Member.unapply(other)
override def toString = "Member(address = %s, status = %s)" format (address, status)
def copy(address: Address = this.address, status: MemberStatus = this.status): Member = new Member(address, status)
}
/**
* Module with factory and ordering methods for Member instances.
*/
object Member {
/**
* `Address` ordering type class, sorts addresses by host and port.
*/
implicit val addressOrdering: Ordering[Address] = Ordering.fromLessThan[Address] { (a, b)
if (a.host != b.host) a.host.getOrElse("").compareTo(b.host.getOrElse("")) < 0
else if (a.port != b.port) a.port.getOrElse(0) < b.port.getOrElse(0)
else false
}
/**
* `Member` ordering type class, sorts members by host and port with the exception that
* it puts all members that are in MemberStatus.EXITING last.
*/
implicit val ordering: Ordering[Member] = Ordering.fromLessThan[Member] { (a, b)
if (a.status == Exiting && b.status != Exiting) false
else if (a.status != Exiting && b.status == Exiting) true
else addressOrdering.compare(a.address, b.address) < 0
}
def apply(address: Address, status: MemberStatus): Member = new Member(address, status)
def unapply(other: Any) = other match {
case m: Member Some(m.address)
case _ None
}
def pickHighestPriority(a: Set[Member], b: Set[Member]): Set[Member] = {
// group all members by Address => Seq[Member]
val groupedByAddress = (a.toSeq ++ b.toSeq).groupBy(_.address)
// pick highest MemberStatus
(Set.empty[Member] /: groupedByAddress) {
case (acc, (_, members)) acc + members.reduceLeft(highestPriorityOf)
}
}
/**
* Picks the Member with the highest "priority" MemberStatus.
*/
def highestPriorityOf(m1: Member, m2: Member): Member = (m1.status, m2.status) match {
case (Removed, _) m1
case (_, Removed) m2
case (Down, _) m1
case (_, Down) m2
case (Exiting, _) m1
case (_, Exiting) m2
case (Leaving, _) m1
case (_, Leaving) m2
case (Up, Joining) m2
case (Joining, Up) m1
case (Joining, Joining) m1
case (Up, Up) m1
}
// FIXME Workaround for https://issues.scala-lang.org/browse/SI-5986
// SortedSet + and ++ operators replaces existing element
// Use these :+ and :++ operators for the Gossip members
implicit def sortedSetWorkaround(sortedSet: SortedSet[Member]): SortedSetWorkaround = new SortedSetWorkaround(sortedSet)
class SortedSetWorkaround(sortedSet: SortedSet[Member]) {
implicit def :+(elem: Member): SortedSet[Member] = {
if (sortedSet.contains(elem)) sortedSet
else sortedSet + elem
}
implicit def :++(elems: GenTraversableOnce[Member]): SortedSet[Member] =
sortedSet ++ (elems.toSet diff sortedSet)
}
}
/**
* Defines the current status of a cluster member node
*
* Can be one of: Joining, Up, Leaving, Exiting and Down.
*/
sealed trait MemberStatus extends ClusterMessage {
/**
* Using the same notion for 'unavailable' as 'non-convergence': DOWN
*/
def isUnavailable: Boolean = this == Down
}
object MemberStatus {
case object Joining extends MemberStatus
case object Up extends MemberStatus
case object Leaving extends MemberStatus
case object Exiting extends MemberStatus
case object Down extends MemberStatus
case object Removed extends MemberStatus
}

View file

@ -1,150 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor._
import akka.remote._
import akka.routing._
import akka.event.Logging
import scala.collection.immutable.Map
import scala.annotation.tailrec
import java.util.concurrent.atomic.AtomicReference
/**
* Remote connection manager, manages remote connections, e.g. RemoteActorRef's.
*/
class RemoteConnectionManager(
system: ActorSystemImpl,
remote: RemoteActorRefProvider,
failureDetector: AccrualFailureDetector,
initialConnections: Map[Address, ActorRef] = Map.empty[Address, ActorRef])
extends ConnectionManager {
val log = Logging(system, "RemoteConnectionManager")
// FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc.
case class State(version: Long, connections: Map[Address, ActorRef])
extends VersionedIterable[ActorRef] {
def iterable: Iterable[ActorRef] = connections.values
}
private val state: AtomicReference[State] = new AtomicReference[State](newState())
/**
* This method is using the FailureDetector to filter out connections that are considered not available.
*/
private def filterAvailableConnections(current: State): State = {
val availableConnections = current.connections filter { entry failureDetector.isAvailable(entry._1) }
current copy (version = current.version, connections = availableConnections)
}
private def newState() = State(Long.MinValue, initialConnections)
def version: Long = state.get.version
// FIXME should not return State value but a Seq with connections
def connections = filterAvailableConnections(state.get)
def size: Int = connections.connections.size
def connectionFor(address: Address): Option[ActorRef] = connections.connections.get(address)
def isEmpty: Boolean = connections.connections.isEmpty
def shutdown() {
state.get.iterable foreach (system.stop(_)) // shut down all remote connections
}
@tailrec
final def failOver(from: Address, to: Address) {
log.debug("Failing over connection from [{}] to [{}]", from, to)
val oldState = state.get
var changed = false
val newMap = oldState.connections map {
case (`from`, actorRef)
changed = true
//actorRef.stop()
(to, newConnection(to, actorRef.path))
case other other
}
if (changed) {
//there was a state change, so we are now going to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newMap)
//if we are not able to update, the state, we are going to try again.
if (!state.compareAndSet(oldState, newState)) {
failOver(from, to) // recur
}
}
}
@tailrec
final def remove(faultyConnection: ActorRef) {
val oldState = state.get()
var changed = false
var faultyAddress: Address = null
var newConnections = Map.empty[Address, ActorRef]
oldState.connections.keys foreach { address
val actorRef: ActorRef = oldState.connections.get(address).get
if (actorRef ne faultyConnection) {
newConnections = newConnections + ((address, actorRef))
} else {
faultyAddress = address
changed = true
}
}
if (changed) {
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) {
remove(faultyConnection) // recur
} else {
log.debug("Removing connection [{}]", faultyAddress)
}
}
}
@tailrec
final def putIfAbsent(address: Address, newConnectionFactory: () ActorRef): ActorRef = {
val oldState = state.get()
val oldConnections = oldState.connections
oldConnections.get(address) match {
case Some(connection) connection // we already had the connection, return it
case None // we need to create it
val newConnection = newConnectionFactory()
val newConnections = oldConnections + (address -> newConnection)
//one or more occurrances of the actorRef were removed, so we need to update the state.
val newState = oldState copy (version = oldState.version + 1, connections = newConnections)
//if we are not able to update the state, we just try again.
if (!state.compareAndSet(oldState, newState)) {
// we failed, need compensating action
system.stop(newConnection) // stop the new connection actor and try again
putIfAbsent(address, newConnectionFactory) // recur
} else {
// we succeeded
log.debug("Adding connection [{}]", address)
newConnection // return new connection actor
}
}
}
private[cluster] def newConnection(remoteAddress: Address, actorPath: ActorPath) =
new RemoteActorRef(remote, remote.transport, actorPath, Nobody)
}

View file

@ -36,6 +36,10 @@ abstract class JoinSeedNodeSpec
"A cluster with configured seed nodes" must { "A cluster with configured seed nodes" must {
"start the seed nodes sequentially" taggedAs LongRunningTest in { "start the seed nodes sequentially" taggedAs LongRunningTest in {
// without looking up the addresses first there might be
// [akka://JoinSeedNodeSpec/user/TestConductorClient] cannot write GetAddress(RoleName(seed2)) while waiting for seed1
roles foreach address
runOn(seed1) { runOn(seed1) {
startClusterNode() startClusterNode()
} }

View file

@ -36,12 +36,22 @@ object LargeClusterMultiJvmSpec extends MultiNodeConfig {
akka.cluster { akka.cluster {
gossip-interval = 500 ms gossip-interval = 500 ms
auto-join = off auto-join = off
failure-detector.threshold = 4 auto-down = on
failure-detector.acceptable-heartbeat-pause = 10s
publish-state-interval = 0 s # always, when it happens
} }
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.default-dispatcher.fork-join-executor.parallelism-max = 2 akka.actor.default-dispatcher.fork-join-executor {
# when using nodes-per-datacenter=10 we need some extra
# threads to keep up with netty connect blocking
parallelism-min = 13
parallelism-max = 13
}
akka.scheduler.tick-duration = 33 ms akka.scheduler.tick-duration = 33 ms
akka.remote.netty.execution-pool-size = 0 akka.remote.netty.execution-pool-size = 4
#akka.remote.netty.reconnection-time-window = 1s
akka.remote.netty.backoff-timeout = 500ms
akka.remote.netty.connection-timeout = 500ms
# don't use testconductor transport in this test, especially not # don't use testconductor transport in this test, especially not
# when using use-dispatcher-for-io # when using use-dispatcher-for-io
@ -124,8 +134,10 @@ abstract class LargeClusterSpec
val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet) val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet)
val startGossipCounts = Map.empty[Cluster, Long] ++ val startGossipCounts = Map.empty[Cluster, Long] ++
clusterNodes.map(c (c -> c.receivedGossipCount)) clusterNodes.map(c (c -> c.latestStats.receivedGossipCount))
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c) def gossipCount(c: Cluster): Long = {
c.latestStats.receivedGossipCount - startGossipCounts(c)
}
val startTime = System.nanoTime val startTime = System.nanoTime
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms" def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
@ -244,15 +256,16 @@ abstract class LargeClusterSpec
} }
} }
// 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 in {
"detect failure and auto-down crashed nodes in second-datacenter" taggedAs LongRunningTest ignore {
val unreachableNodes = nodesPerDatacenter val unreachableNodes = nodesPerDatacenter
val liveNodes = nodesPerDatacenter * 4 val liveNodes = nodesPerDatacenter * 4
within(20.seconds + expectedMaxDuration(liveNodes)) { within(30.seconds + (3.seconds * liveNodes)) {
val startGossipCounts = Map.empty[Cluster, Long] ++ val startGossipCounts = Map.empty[Cluster, Long] ++
systems.map(sys (Cluster(sys) -> Cluster(sys).receivedGossipCount)) systems.map(sys (Cluster(sys) -> Cluster(sys).latestStats.receivedGossipCount))
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c) def gossipCount(c: Cluster): Long = {
c.latestStats.receivedGossipCount - startGossipCounts(c)
}
val startTime = System.nanoTime val startTime = System.nanoTime
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms" def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
@ -278,10 +291,11 @@ abstract class LargeClusterSpec
runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) { runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) {
Await.ready(latch, remaining) Await.ready(latch, remaining)
awaitCond(systems.forall(Cluster(_).convergence.isDefined)) awaitCond(systems.forall(Cluster(_).convergence.isDefined))
val mergeCount = systems.map(sys Cluster(sys).latestStats.mergeCount).sum
val counts = systems.map(sys gossipCount(Cluster(sys))) val counts = systems.map(sys gossipCount(Cluster(sys)))
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / nodesPerDatacenter, counts.min, counts.max) 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", log.info("Convergence of [{}] nodes reached after failure, it took [{}], received [{}] gossip messages per node, merged [{}] times",
liveNodes, tookMillis, formattedStats) liveNodes, tookMillis, formattedStats, mergeCount)
} }
enterBarrier("after-6") enterBarrier("after-6")

View file

@ -27,6 +27,7 @@ object MultiNodeClusterSpec {
leader-actions-interval = 200 ms leader-actions-interval = 200 ms
unreachable-nodes-reaper-interval = 200 ms unreachable-nodes-reaper-interval = 200 ms
periodic-tasks-initial-delay = 300 ms periodic-tasks-initial-delay = 300 ms
publish-state-interval = 0 s # always, when it happens
} }
akka.test { akka.test {
single-expect-default = 5 s single-expect-default = 5 s

View file

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

View file

@ -38,7 +38,7 @@ abstract class NodeUpSpec
"be unaffected when joining again" taggedAs LongRunningTest in { "be unaffected when joining again" taggedAs LongRunningTest in {
val unexpected = new AtomicReference[SortedSet[Member]] val unexpected = new AtomicReference[SortedSet[Member]](SortedSet.empty)
cluster.registerListener(new MembershipChangeListener { cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) { def notify(members: SortedSet[Member]) {
if (members.size != 2 || members.exists(_.status != MemberStatus.Up)) if (members.size != 2 || members.exists(_.status != MemberStatus.Up))
@ -55,7 +55,7 @@ abstract class NodeUpSpec
// let it run for a while to make sure that nothing bad happens // let it run for a while to make sure that nothing bad happens
for (n 1 to 20) { for (n 1 to 20) {
100.millis.dilated.sleep() 100.millis.dilated.sleep()
unexpected.get must be(null) unexpected.get must be(SortedSet.empty)
cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true) cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true)
} }

View file

@ -9,8 +9,10 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.actor.Address import akka.actor.Address
import akka.pattern.ask
import akka.remote.testconductor.RoleName import akka.remote.testconductor.RoleName
import MemberStatus._ import MemberStatus._
import InternalClusterAction._
object TransitionMultiJvmSpec extends MultiNodeConfig { object TransitionMultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -28,7 +30,8 @@ class TransitionMultiJvmNode3 extends TransitionSpec with FailureDetectorPuppetS
abstract class TransitionSpec abstract class TransitionSpec
extends MultiNodeSpec(TransitionMultiJvmSpec) extends MultiNodeSpec(TransitionMultiJvmSpec)
with MultiNodeClusterSpec { with MultiNodeClusterSpec
with ImplicitSender {
import TransitionMultiJvmSpec._ import TransitionMultiJvmSpec._
@ -67,6 +70,22 @@ abstract class TransitionSpec
memberStatus(address) == status memberStatus(address) == status
} }
def leaderActions(): Unit = {
cluster.clusterCore ! LeaderActionsTick
awaitPing()
}
def reapUnreachable(): Unit = {
cluster.clusterCore ! ReapUnreachableTick
awaitPing()
}
def awaitPing(): Unit = {
val ping = Ping()
cluster.clusterCore ! ping
expectMsgPF() { case pong @ Pong(`ping`, _) pong }
}
// DSL sugar for `role1 gossipTo role2` // DSL sugar for `role1 gossipTo role2`
implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role) implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role)
var gossipBarrierCounter = 0 var gossipBarrierCounter = 0
@ -83,7 +102,8 @@ abstract class TransitionSpec
} }
runOn(fromRole) { runOn(fromRole) {
enterBarrier("before-gossip-" + gossipBarrierCounter) enterBarrier("before-gossip-" + gossipBarrierCounter)
cluster.gossipTo(toRole) // send gossip // send gossip
cluster.clusterCore ! InternalClusterAction.SendGossipTo(toRole)
// gossip chat will synchronize the views // gossip chat will synchronize the views
awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty) awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty)
enterBarrier("after-gossip-" + gossipBarrierCounter) enterBarrier("after-gossip-" + gossipBarrierCounter)
@ -104,7 +124,7 @@ abstract class TransitionSpec
cluster.isSingletonCluster must be(true) cluster.isSingletonCluster must be(true)
cluster.status must be(Joining) cluster.status must be(Joining)
cluster.convergence.isDefined must be(true) cluster.convergence.isDefined must be(true)
cluster.leaderActions() leaderActions()
cluster.status must be(Up) cluster.status must be(Up)
} }
@ -127,7 +147,7 @@ abstract class TransitionSpec
enterBarrier("convergence-joining-2") enterBarrier("convergence-joining-2")
runOn(leader(first, second)) { runOn(leader(first, second)) {
cluster.leaderActions() leaderActions()
memberStatus(first) must be(Up) memberStatus(first) must be(Up)
memberStatus(second) must be(Up) memberStatus(second) must be(Up)
} }
@ -182,7 +202,7 @@ abstract class TransitionSpec
enterBarrier("convergence-joining-3") enterBarrier("convergence-joining-3")
runOn(leader(first, second, third)) { runOn(leader(first, second, third)) {
cluster.leaderActions() leaderActions()
memberStatus(first) must be(Up) memberStatus(first) must be(Up)
memberStatus(second) must be(Up) memberStatus(second) must be(Up)
memberStatus(third) must be(Up) memberStatus(third) must be(Up)
@ -200,7 +220,8 @@ abstract class TransitionSpec
// first non-leader gossipTo the other non-leader // first non-leader gossipTo the other non-leader
nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head
runOn(nonLeader(first, second, third).head) { runOn(nonLeader(first, second, third).head) {
cluster.gossipTo(nonLeader(first, second, third).tail.head) // send gossip
cluster.clusterCore ! InternalClusterAction.SendGossipTo(nonLeader(first, second, third).tail.head)
} }
runOn(nonLeader(first, second, third).tail.head) { runOn(nonLeader(first, second, third).tail.head) {
memberStatus(third) must be(Up) memberStatus(third) must be(Up)
@ -224,7 +245,7 @@ abstract class TransitionSpec
"perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in { "perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in {
runOn(third) { runOn(third) {
markNodeAsUnavailable(second) markNodeAsUnavailable(second)
cluster.reapUnreachableMembers() reapUnreachable()
cluster.latestGossip.overview.unreachable must contain(Member(second, Up)) cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
seenLatestGossip must be(Set(third)) seenLatestGossip must be(Set(third))
} }

View file

@ -9,7 +9,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.Address import akka.actor.Address
import akka.remote.testconductor.{RoleName, Direction} import akka.remote.testconductor.{ RoleName, Direction }
import akka.util.duration._ import akka.util.duration._
object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig { object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig {
@ -26,7 +26,6 @@ class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode2 extend
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
@ -41,7 +40,6 @@ abstract class UnreachableNodeRejoinsClusterSpec
roles.filterNot(_ == role) roles.filterNot(_ == role)
} }
lazy val sortedRoles = roles.sorted lazy val sortedRoles = roles.sorted
lazy val master = sortedRoles(0) lazy val master = sortedRoles(0)
lazy val victim = sortedRoles(1) lazy val victim = sortedRoles(1)
@ -55,14 +53,19 @@ abstract class UnreachableNodeRejoinsClusterSpec
"A cluster of " + roles.size + " members" must { "A cluster of " + roles.size + " members" must {
"reach initial convergence" taggedAs LongRunningTest in { "reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(roles:_*) awaitClusterUp(roles: _*)
endBarrier endBarrier
} }
"mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in { "mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in {
// let them send at least one heartbeat to each other after the gossip convergence
// because for new joining nodes we remove them from the failure detector when
// receive gossip
2.seconds.dilated.sleep
runOn(first) { runOn(first) {
// pull network for victim node from all nodes // pull network for victim node from all nodes
allBut(victim).foreach { roleName => allBut(victim).foreach { roleName
testConductor.blackhole(victim, roleName, Direction.Both).await testConductor.blackhole(victim, roleName, Direction.Both).await
} }
} }
@ -74,24 +77,28 @@ abstract class UnreachableNodeRejoinsClusterSpec
allButVictim.foreach(markNodeAsUnavailable(_)) allButVictim.foreach(markNodeAsUnavailable(_))
within(30 seconds) { within(30 seconds) {
// victim becomes all alone // victim becomes all alone
awaitCond({ val gossip = cluster.latestGossip awaitCond({
val gossip = cluster.latestGossip
gossip.overview.unreachable.size == (roles.size - 1) && gossip.overview.unreachable.size == (roles.size - 1) &&
gossip.members.size == 1 && gossip.members.size == 1 &&
gossip.members.forall(_.status == MemberStatus.Up) }) gossip.members.forall(_.status == MemberStatus.Up)
})
cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet) cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet)
cluster.convergence.isDefined must be(false) cluster.convergence.isDefined must be(false)
} }
} }
runOn(allButVictim:_*) { runOn(allButVictim: _*) {
markNodeAsUnavailable(victim) markNodeAsUnavailable(victim)
within(30 seconds) { within(30 seconds) {
// victim becomes unreachable // victim becomes unreachable
awaitCond({ val gossip = cluster.latestGossip awaitCond({
val gossip = cluster.latestGossip
gossip.overview.unreachable.size == 1 && gossip.overview.unreachable.size == 1 &&
gossip.members.size == (roles.size - 1) && gossip.members.size == (roles.size - 1) &&
gossip.members.forall(_.status == MemberStatus.Up) }) gossip.members.forall(_.status == MemberStatus.Up)
awaitSeenSameState(allButVictim map address:_*) })
awaitSeenSameState(allButVictim map address: _*)
// still one unreachable // still one unreachable
cluster.latestGossip.overview.unreachable.size must be(1) cluster.latestGossip.overview.unreachable.size must be(1)
cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address) cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address)
@ -108,7 +115,7 @@ abstract class UnreachableNodeRejoinsClusterSpec
cluster down victim cluster down victim
} }
runOn(allBut(victim):_*) { runOn(allBut(victim): _*) {
awaitUpConvergence(roles.size - 1, Seq(victim)) awaitUpConvergence(roles.size - 1, Seq(victim))
} }
@ -118,7 +125,7 @@ abstract class UnreachableNodeRejoinsClusterSpec
"allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in { "allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in {
runOn(first) { runOn(first) {
// put the network back in // put the network back in
allBut(victim).foreach { roleName => allBut(victim).foreach { roleName
testConductor.passThrough(victim, roleName, Direction.Both).await testConductor.passThrough(victim, roleName, Direction.Both).await
} }
} }

View file

@ -7,6 +7,7 @@ package akka.cluster
import language.postfixOps import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.dispatch.Dispatchers
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
@ -30,13 +31,19 @@ class ClusterConfigSpec extends AkkaSpec {
HeartbeatInterval must be(1 second) HeartbeatInterval must be(1 second)
LeaderActionsInterval must be(1 second) LeaderActionsInterval must be(1 second)
UnreachableNodesReaperInterval must be(1 second) UnreachableNodesReaperInterval must be(1 second)
PublishStateInterval must be(1 second)
JoinTimeout must be(60 seconds) JoinTimeout must be(60 seconds)
NrOfGossipDaemons must be(4)
AutoJoin must be(true) AutoJoin must be(true)
AutoDown must be(false) AutoDown must be(false)
UseDispatcher must be(Dispatchers.DefaultDispatcherId)
GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001) GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001)
MaxGossipMergeRate must be(5.0 plusOrMinus 0.0001)
SchedulerTickDuration must be(33 millis) SchedulerTickDuration must be(33 millis)
SchedulerTicksPerWheel must be(512) SchedulerTicksPerWheel must be(512)
SendCircuitBreakerSettings must be(CircuitBreakerSettings(
maxFailures = 3,
callTimeout = 2 seconds,
resetTimeout = 30 seconds))
} }
} }
} }

View file

@ -7,14 +7,17 @@ package akka.cluster
import language.postfixOps import language.postfixOps
import language.reflectiveCalls import language.reflectiveCalls
import akka.testkit.AkkaSpec
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.Address import akka.actor.Address
import java.util.concurrent.atomic.AtomicInteger import akka.cluster.InternalClusterAction._
import org.scalatest.BeforeAndAfter
import akka.remote.RemoteActorRefProvider import akka.remote.RemoteActorRefProvider
import java.lang.management.ManagementFactory
import javax.management.ObjectName
object ClusterSpec { object ClusterSpec {
val config = """ val config = """
@ -22,6 +25,7 @@ object ClusterSpec {
auto-join = off auto-join = off
auto-down = off auto-down = off
periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks
publish-state-interval = 0 s # always, when it happens
} }
akka.actor.provider = "akka.remote.RemoteActorRefProvider" akka.actor.provider = "akka.remote.RemoteActorRefProvider"
akka.remote.netty.port = 0 akka.remote.netty.port = 0
@ -32,57 +36,24 @@ object ClusterSpec {
} }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter { class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
import ClusterSpec._ import ClusterSpec._
val selfAddress = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.address 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 failureDetector = new FailureDetectorPuppet(system) val failureDetector = new FailureDetectorPuppet(system)
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) { 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
else Some(addresses.toSeq(deterministicRandom.getAndIncrement % addresses.size))
}
override def gossipTo(address: Address): Unit = {
if (address == self.address) {
super.gossipTo(address)
}
// represent the gossip with a message to be used in asserts
testActor ! GossipTo(address)
}
@volatile
var _gossipToDeputyProbablity = 0.0
override def gossipToDeputyProbablity(membersSize: Int, unreachableSize: Int, deputySize: Int): Double = {
if (_gossipToDeputyProbablity < 0.0) super.gossipToDeputyProbablity(membersSize, unreachableSize, deputySize)
else _gossipToDeputyProbablity
}
def leaderActions(): Unit = {
cluster.clusterCore ! LeaderActionsTick
awaitPing()
} }
def memberStatus(address: Address): Option[MemberStatus] = def awaitPing(): Unit = {
cluster.latestGossip.members.collectFirst { case m if m.address == address m.status } val ping = Ping()
cluster.clusterCore ! ping
before { expectMsgPF() { case pong @ Pong(`ping`, _) pong }
cluster._gossipToDeputyProbablity = 0.0
addresses foreach failureDetector.remove
deterministicRandom.set(0)
} }
"A Cluster" must { "A Cluster" must {
@ -91,106 +62,24 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
cluster.selfAddress must be(selfAddress) cluster.selfAddress must be(selfAddress)
} }
"register jmx mbean" in {
val name = new ObjectName("akka:type=Cluster")
val info = ManagementFactory.getPlatformMBeanServer.getMBeanInfo(name)
info.getAttributes.length must be > (0)
info.getOperations.length must be > (0)
}
"initially become singleton cluster when joining itself and reach convergence" in { "initially become singleton cluster when joining itself and reach convergence" in {
cluster.isSingletonCluster must be(false) // auto-join = off cluster.isSingletonCluster must be(false) // auto-join = off
cluster.join(selfAddress) cluster.join(selfAddress)
awaitCond(cluster.isSingletonCluster) awaitCond(cluster.isSingletonCluster)
cluster.self.address must be(selfAddress) cluster.self.address must be(selfAddress)
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress)) cluster.latestGossip.members.map(_.address) must be(Set(selfAddress))
memberStatus(selfAddress) must be(Some(MemberStatus.Joining)) cluster.status must be(MemberStatus.Joining)
cluster.convergence.isDefined must be(true) cluster.convergence.isDefined must be(true)
cluster.leaderActions() leaderActions()
memberStatus(selfAddress) must be(Some(MemberStatus.Up)) cluster.status must be(MemberStatus.Up)
} }
"accept a joining node" in {
cluster.joining(addresses(1))
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress, addresses(1)))
memberStatus(addresses(1)) must be(Some(MemberStatus.Joining))
cluster.convergence.isDefined must be(false)
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)
}
"order members by host and port" in {
// note the importance of using toSeq before map, otherwise it will not preserve the order
cluster.latestGossip.members.toSeq.map(_.address) must be(addresses.toSeq)
}
"gossip to random live node" in {
cluster.gossip()
cluster.gossip()
cluster.gossip()
cluster.gossip()
expectMsg(GossipTo(addresses(1)))
expectMsg(GossipTo(addresses(2)))
expectMsg(GossipTo(addresses(3)))
expectMsg(GossipTo(addresses(4)))
expectNoMsg(1 second)
}
"use certain probability for gossiping to deputy node depending on the number of unreachable and live nodes" in {
cluster._gossipToDeputyProbablity = -1.0 // use real impl
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(9, 1, 2))
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 2))
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 3))
cluster.gossipToDeputyProbablity(10, 5, 5) must be < (cluster.gossipToDeputyProbablity(10, 9, 5))
cluster.gossipToDeputyProbablity(0, 10, 0) must be <= (1.0)
cluster.gossipToDeputyProbablity(1, 10, 1) must be <= (1.0)
cluster.gossipToDeputyProbablity(10, 0, 0) must be(0.0 plusOrMinus (0.0001))
cluster.gossipToDeputyProbablity(0, 0, 0) must be(0.0 plusOrMinus (0.0001))
cluster.gossipToDeputyProbablity(4, 0, 4) must be(1.0 plusOrMinus (0.0001))
cluster.gossipToDeputyProbablity(3, 7, 4) must be(1.0 plusOrMinus (0.0001))
}
"gossip to duputy node" in {
cluster._gossipToDeputyProbablity = 1.0 // always
// we have configured 3 deputy nodes (seedNodes)
cluster.gossip() // 1 is deputy
cluster.gossip() // 2 is deputy
cluster.gossip() // 3 is deputy
cluster.gossip() // 4 is not deputy, and therefore a deputy is also used
expectMsg(GossipTo(addresses(1)))
expectMsg(GossipTo(addresses(2)))
expectMsg(GossipTo(addresses(3)))
expectMsg(GossipTo(addresses(4)))
// and the extra gossip to deputy
expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3)))
expectNoMsg(1 second)
}
"gossip to random deputy node if number of live nodes is less than number of deputy nodes" in {
cluster._gossipToDeputyProbablity = -1.0 // real impl
// 0 and 2 still alive
val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5))
dead foreach failureDetector.markNodeAsUnavailable
cluster.reapUnreachableMembers()
cluster.latestGossip.overview.unreachable.map(_.address) must be(dead)
for (n 1 to 20) {
cluster.gossip()
expectMsg(GossipTo(addresses(2))) // the only available
// and always to one of the 3 deputies
expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3)))
}
expectNoMsg(1 second)
}
} }
} }

View file

@ -57,9 +57,6 @@ These terms are used throughout the documentation.
A single node in the cluster that acts as the leader. Managing cluster convergence, A single node in the cluster that acts as the leader. Managing cluster convergence,
partitions, fail-over, rebalancing etc. partitions, fail-over, rebalancing etc.
**deputy nodes**
A set of nodes responsible for breaking logical partitions.
Membership Membership
========== ==========
@ -192,16 +189,6 @@ then sends join command to the one that answers first.
It is possible to turn off automatic join. It is possible to turn off automatic join.
Deputy Nodes
^^^^^^^^^^^^
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
algorithm defined below.
Gossip Protocol Gossip Protocol
^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^
@ -219,26 +206,18 @@ nodes involved in a gossip exchange.
Periodically, the default is every 1 second, each node chooses another random 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 node to initiate a round of gossip with. The choice of node is random but can
also include extra gossiping for ``deputy`` nodes, and nodes with also include extra gossiping nodes with either newer or older state versions.
either newer or older state versions.
The gossip overview contains the current state version for all nodes and also a The gossip overview contains the current state version for all nodes and also a
list of unreachable nodes. Whenever a node receives a gossip overview it updates list of unreachable nodes. Whenever a node receives a gossip overview it updates
the `Failure Detector`_ with the liveness information. the `Failure Detector`_ with the liveness information.
The nodes defined as ``deputy`` nodes are just regular member nodes whose only The nodes defined as ``seed`` nodes are just regular member nodes whose only
"special role" is to function as contact points in the cluster and to help "special role" is to function as contact points in the cluster.
breaking logical partitions as seen in the gossip algorithm defined below.
During each round of gossip exchange the following process is used: During each round of gossip exchange it sends Gossip to random node with
newer or older state information, if any, based on the current gossip overview,
1. Gossip to random node with newer or older state information, if any, based on the with some probability. Otherwise Gossip to any random live node.
current gossip overview, with some probability. Otherwise Gossip to any random
live node.
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.
The gossiper only sends the gossip overview to the chosen node. The recipient of The gossiper only sends the gossip overview to the chosen node. The recipient of
the gossip can use the gossip overview to determine whether: the gossip can use the gossip overview to determine whether:

View file

@ -12,7 +12,7 @@ import scala.concurrent.util.Duration;
import akka.pattern.CircuitBreaker; import akka.pattern.CircuitBreaker;
import akka.event.Logging; import akka.event.Logging;
import static scala.concurrent.Futures.future; import static akka.dispatch.Futures.future;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;

View file

@ -6,10 +6,11 @@ package docs.actor;
//#imports //#imports
import akka.actor.TypedActor; import akka.actor.TypedActor;
import akka.dispatch.*;
import akka.actor.*; import akka.actor.*;
import akka.japi.*; import akka.japi.*;
import akka.dispatch.Futures;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;

View file

@ -11,7 +11,7 @@ import akka.actor.Props;
//#import-future //#import-future
import scala.concurrent.Future; import scala.concurrent.Future;
import scala.concurrent.Futures; import akka.dispatch.Futures;
import akka.dispatch.Mapper; import akka.dispatch.Mapper;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
@ -43,13 +43,17 @@ import akka.pattern.AskTimeoutException;
import static akka.pattern.Patterns.ask; import static akka.pattern.Patterns.ask;
import static akka.pattern.Patterns.pipe; import static akka.pattern.Patterns.pipe;
import scala.concurrent.Future; import scala.concurrent.Future;
import scala.concurrent.Futures; import akka.dispatch.Futures;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
import akka.util.Timeout; import akka.util.Timeout;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.ArrayList; import java.util.ArrayList;
//#import-askPipe //#import-askPipe
//#import-stash
import akka.actor.UntypedActorWithStash;
//#import-stash
import akka.actor.Props; import akka.actor.Props;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
@ -246,7 +250,7 @@ public class UntypedActorDocTestBase {
final int x = (Integer) it.next(); final int x = (Integer) it.next();
return new Result(x, s); return new Result(x, s);
} }
}); }, system.dispatcher());
pipe(transformed).to(actorC); pipe(transformed).to(actorC);
//#ask-pipe //#ask-pipe
@ -346,6 +350,31 @@ public class UntypedActorDocTestBase {
//#hot-swap-actor //#hot-swap-actor
//#stash
public static class ActorWithProtocol extends UntypedActorWithStash {
private Boolean isOpen = false;
public void onReceive(Object msg) {
if (isOpen) {
if (msg.equals("write")) {
// do writing...
} else if (msg.equals("close")) {
unstashAll();
isOpen = false;
} else {
stash();
}
} else {
if (msg.equals("open")) {
unstashAll();
isOpen = true;
} else {
stash();
}
}
}
}
//#stash
//#watch //#watch
public static class WatchActor extends UntypedActor { public static class WatchActor extends UntypedActor {
final ActorRef child = this.getContext().actorOf(Props.empty(), "child"); final ActorRef child = this.getContext().actorOf(Props.empty(), "child");

View file

@ -151,7 +151,7 @@ public class FaultHandlingDocSample {
public Progress apply(CurrentCount c) { public Progress apply(CurrentCount c) {
return new Progress(100.0 * c.count / totalCount); return new Progress(100.0 * c.count / totalCount);
} }
})) }, getContext().dispatcher()))
.to(progressListener); .to(progressListener);
} else { } else {
unhandled(msg); unhandled(msg);

View file

@ -5,6 +5,8 @@ package docs.future;
//#imports1 //#imports1
import akka.dispatch.*; import akka.dispatch.*;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.concurrent.Await; import scala.concurrent.Await;
import akka.util.Timeout; import akka.util.Timeout;
@ -14,29 +16,29 @@ import akka.util.Timeout;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
import akka.japi.Function; import akka.japi.Function;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import static scala.concurrent.Futures.future; import static akka.dispatch.Futures.future;
import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.concurrent.TimeUnit.SECONDS;
//#imports2 //#imports2
//#imports3 //#imports3
import static scala.concurrent.Futures.sequence; import static akka.dispatch.Futures.sequence;
//#imports3 //#imports3
//#imports4 //#imports4
import static scala.concurrent.Futures.traverse; import static akka.dispatch.Futures.traverse;
//#imports4 //#imports4
//#imports5 //#imports5
import akka.japi.Function2; import akka.japi.Function2;
import static scala.concurrent.Futures.fold; import static akka.dispatch.Futures.fold;
//#imports5 //#imports5
//#imports6 //#imports6
import static scala.concurrent.Futures.reduce; import static akka.dispatch.Futures.reduce;
//#imports6 //#imports6
@ -122,17 +124,19 @@ public class FutureDocTestBase {
@Test @Test
public void useMap() throws Exception { public void useMap() throws Exception {
//#map //#map
final ExecutionContext ec = system.dispatcher();
Future<String> f1 = future(new Callable<String>() { Future<String> f1 = future(new Callable<String>() {
public String call() { public String call() {
return "Hello" + "World"; return "Hello" + "World";
} }
}, system.dispatcher()); }, ec);
Future<Integer> f2 = f1.map(new Mapper<String, Integer>() { Future<Integer> f2 = f1.map(new Mapper<String, Integer>() {
public Integer apply(String s) { public Integer apply(String s) {
return s.length(); return s.length();
} }
}); }, ec);
int result = Await.result(f2, Duration.create(1, SECONDS)); int result = Await.result(f2, Duration.create(1, SECONDS));
assertEquals(10, result); assertEquals(10, result);
@ -142,18 +146,20 @@ public class FutureDocTestBase {
@Test @Test
public void useMap2() throws Exception { public void useMap2() throws Exception {
//#map2 //#map2
final ExecutionContext ec = system.dispatcher();
Future<String> f1 = future(new Callable<String>() { Future<String> f1 = future(new Callable<String>() {
public String call() throws Exception { public String call() throws Exception {
Thread.sleep(100); Thread.sleep(100);
return "Hello" + "World"; return "Hello" + "World";
} }
}, system.dispatcher()); }, ec);
Future<Integer> f2 = f1.map(new Mapper<String, Integer>() { Future<Integer> f2 = f1.map(new Mapper<String, Integer>() {
public Integer apply(String s) { public Integer apply(String s) {
return s.length(); return s.length();
} }
}); }, ec);
//#map2 //#map2
int result = Await.result(f2, Duration.create(1, SECONDS)); int result = Await.result(f2, Duration.create(1, SECONDS));
@ -163,11 +169,13 @@ public class FutureDocTestBase {
@Test @Test
public void useMap3() throws Exception { public void useMap3() throws Exception {
//#map3 //#map3
final ExecutionContext ec = system.dispatcher();
Future<String> f1 = future(new Callable<String>() { Future<String> f1 = future(new Callable<String>() {
public String call() { public String call() {
return "Hello" + "World"; return "Hello" + "World";
} }
}, system.dispatcher()); }, ec);
Thread.sleep(100); Thread.sleep(100);
@ -175,7 +183,7 @@ public class FutureDocTestBase {
public Integer apply(String s) { public Integer apply(String s) {
return s.length(); return s.length();
} }
}); }, ec);
//#map3 //#map3
int result = Await.result(f2, Duration.create(1, SECONDS)); int result = Await.result(f2, Duration.create(1, SECONDS));
@ -185,11 +193,13 @@ public class FutureDocTestBase {
@Test @Test
public void useFlatMap() throws Exception { public void useFlatMap() throws Exception {
//#flat-map //#flat-map
final ExecutionContext ec = system.dispatcher();
Future<String> f1 = future(new Callable<String>() { Future<String> f1 = future(new Callable<String>() {
public String call() { public String call() {
return "Hello" + "World"; return "Hello" + "World";
} }
}, system.dispatcher()); }, ec);
Future<Integer> f2 = f1.flatMap(new Mapper<String, Future<Integer>>() { Future<Integer> f2 = f1.flatMap(new Mapper<String, Future<Integer>>() {
public Future<Integer> apply(final String s) { public Future<Integer> apply(final String s) {
@ -197,9 +207,9 @@ public class FutureDocTestBase {
public Integer call() { public Integer call() {
return s.length(); return s.length();
} }
}, system.dispatcher()); }, ec);
} }
}); }, ec);
//#flat-map //#flat-map
int result = Await.result(f2, Duration.create(1, SECONDS)); int result = Await.result(f2, Duration.create(1, SECONDS));
@ -213,11 +223,12 @@ public class FutureDocTestBase {
source.add(Futures.successful(2, system.dispatcher())); source.add(Futures.successful(2, system.dispatcher()));
//#sequence //#sequence
final ExecutionContext ec = system.dispatcher();
//Some source generating a sequence of Future<Integer>:s //Some source generating a sequence of Future<Integer>:s
Iterable<Future<Integer>> listOfFutureInts = source; Iterable<Future<Integer>> listOfFutureInts = source;
// now we have a Future[Iterable[Integer]] // now we have a Future[Iterable[Integer]]
Future<Iterable<Integer>> futureListOfInts = sequence(listOfFutureInts, system.dispatcher()); Future<Iterable<Integer>> futureListOfInts = sequence(listOfFutureInts, ec);
// Find the sum of the odd numbers // Find the sum of the odd numbers
Future<Long> futureSum = futureListOfInts.map(new Mapper<Iterable<Integer>, Long>() { Future<Long> futureSum = futureListOfInts.map(new Mapper<Iterable<Integer>, Long>() {
@ -227,7 +238,7 @@ public class FutureDocTestBase {
sum += i; sum += i;
return sum; return sum;
} }
}); }, ec);
long result = Await.result(futureSum, Duration.create(1, SECONDS)); long result = Await.result(futureSum, Duration.create(1, SECONDS));
//#sequence //#sequence
@ -237,6 +248,7 @@ public class FutureDocTestBase {
@Test @Test
public void useTraverse() throws Exception { public void useTraverse() throws Exception {
//#traverse //#traverse
final ExecutionContext ec = system.dispatcher();
//Just a sequence of Strings //Just a sequence of Strings
Iterable<String> listStrings = Arrays.asList("a", "b", "c"); Iterable<String> listStrings = Arrays.asList("a", "b", "c");
@ -246,9 +258,9 @@ public class FutureDocTestBase {
public String call() { public String call() {
return r.toUpperCase(); return r.toUpperCase();
} }
}, system.dispatcher()); }, ec);
} }
}, system.dispatcher()); }, ec);
//Returns the sequence of strings as upper case //Returns the sequence of strings as upper case
Iterable<String> result = Await.result(futureResult, Duration.create(1, SECONDS)); Iterable<String> result = Await.result(futureResult, Duration.create(1, SECONDS));
@ -263,6 +275,8 @@ public class FutureDocTestBase {
source.add(Futures.successful("b", system.dispatcher())); source.add(Futures.successful("b", system.dispatcher()));
//#fold //#fold
final ExecutionContext ec = system.dispatcher();
//A sequence of Futures, in this case Strings //A sequence of Futures, in this case Strings
Iterable<Future<String>> futures = source; Iterable<Future<String>> futures = source;
@ -271,7 +285,7 @@ public class FutureDocTestBase {
public String apply(String r, String t) { public String apply(String r, String t) {
return r + t; //Just concatenate return r + t; //Just concatenate
} }
}, system.dispatcher()); }, ec);
String result = Await.result(resultFuture, Duration.create(1, SECONDS)); String result = Await.result(resultFuture, Duration.create(1, SECONDS));
//#fold //#fold
@ -285,6 +299,7 @@ public class FutureDocTestBase {
source.add(Futures.successful("b", system.dispatcher())); source.add(Futures.successful("b", system.dispatcher()));
//#reduce //#reduce
final ExecutionContext ec = system.dispatcher();
//A sequence of Futures, in this case Strings //A sequence of Futures, in this case Strings
Iterable<Future<String>> futures = source; Iterable<Future<String>> futures = source;
@ -292,7 +307,7 @@ public class FutureDocTestBase {
public Object apply(Object r, String t) { public Object apply(Object r, String t) {
return r + t; //Just concatenate return r + t; //Just concatenate
} }
}, system.dispatcher()); }, ec);
Object result = Await.result(resultFuture, Duration.create(1, SECONDS)); Object result = Await.result(resultFuture, Duration.create(1, SECONDS));
//#reduce //#reduce
@ -302,11 +317,12 @@ public class FutureDocTestBase {
@Test @Test
public void useSuccessfulAndFailed() throws Exception { public void useSuccessfulAndFailed() throws Exception {
final ExecutionContext ec = system.dispatcher();
//#successful //#successful
Future<String> future = Futures.successful("Yay!", system.dispatcher()); Future<String> future = Futures.successful("Yay!", ec);
//#successful //#successful
//#failed //#failed
Future<String> otherFuture = Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher()); Future<String> otherFuture = Futures.failed(new IllegalArgumentException("Bang!"), ec);
//#failed //#failed
Object result = Await.result(future, Duration.create(1, SECONDS)); Object result = Await.result(future, Duration.create(1, SECONDS));
assertEquals("Yay!", result); assertEquals("Yay!", result);
@ -317,18 +333,19 @@ public class FutureDocTestBase {
@Test @Test
public void useFilter() throws Exception { public void useFilter() throws Exception {
//#filter //#filter
Future<Integer> future1 = Futures.successful(4, system.dispatcher()); final ExecutionContext ec = system.dispatcher();
Future<Integer> future1 = Futures.successful(4, ec);
Future<Integer> successfulFilter = future1.filter(Filter.filterOf(new Function<Integer, Boolean>() { Future<Integer> successfulFilter = future1.filter(Filter.filterOf(new Function<Integer, Boolean>() {
public Boolean apply(Integer i) { public Boolean apply(Integer i) {
return i % 2 == 0; return i % 2 == 0;
} }
})); }), ec);
Future<Integer> failedFilter = future1.filter(Filter.filterOf(new Function<Integer, Boolean>() { Future<Integer> failedFilter = future1.filter(Filter.filterOf(new Function<Integer, Boolean>() {
public Boolean apply(Integer i) { public Boolean apply(Integer i) {
return i % 2 != 0; return i % 2 != 0;
} }
})); }), ec);
//When filter fails, the returned Future will be failed with a scala.MatchError //When filter fails, the returned Future will be failed with a scala.MatchError
//#filter //#filter
} }
@ -344,35 +361,38 @@ public class FutureDocTestBase {
@Test @Test
public void useAndThen() { public void useAndThen() {
//#and-then //#and-then
Future<String> future1 = Futures.successful("value", system.dispatcher()).andThen(new OnComplete<String>() { final ExecutionContext ec = system.dispatcher();
Future<String> future1 = Futures.successful("value", ec).andThen(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) { public void onComplete(Throwable failure, String result) {
if (failure != null) if (failure != null)
sendToIssueTracker(failure); sendToIssueTracker(failure);
} }
}).andThen(new OnComplete<String>() { }, ec).andThen(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) { public void onComplete(Throwable failure, String result) {
if (result != null) if (result != null)
sendToTheInternetz(result); sendToTheInternetz(result);
} }
}); }, ec);
//#and-then //#and-then
} }
@Test @Test
public void useRecover() throws Exception { public void useRecover() throws Exception {
//#recover //#recover
final ExecutionContext ec = system.dispatcher();
Future<Integer> future = future(new Callable<Integer>() { Future<Integer> future = future(new Callable<Integer>() {
public Integer call() { public Integer call() {
return 1 / 0; return 1 / 0;
} }
}, system.dispatcher()).recover(new Recover<Integer>() { }, ec).recover(new Recover<Integer>() {
public Integer recover(Throwable problem) throws Throwable { public Integer recover(Throwable problem) throws Throwable {
if (problem instanceof ArithmeticException) if (problem instanceof ArithmeticException)
return 0; return 0;
else else
throw problem; throw problem;
} }
}); }, ec);
int result = Await.result(future, Duration.create(1, SECONDS)); int result = Await.result(future, Duration.create(1, SECONDS));
assertEquals(result, 0); assertEquals(result, 0);
//#recover //#recover
@ -381,22 +401,24 @@ public class FutureDocTestBase {
@Test @Test
public void useTryRecover() throws Exception { public void useTryRecover() throws Exception {
//#try-recover //#try-recover
final ExecutionContext ec = system.dispatcher();
Future<Integer> future = future(new Callable<Integer>() { Future<Integer> future = future(new Callable<Integer>() {
public Integer call() { public Integer call() {
return 1 / 0; return 1 / 0;
} }
}, system.dispatcher()).recoverWith(new Recover<Future<Integer>>() { }, ec).recoverWith(new Recover<Future<Integer>>() {
public Future<Integer> recover(Throwable problem) throws Throwable { public Future<Integer> recover(Throwable problem) throws Throwable {
if (problem instanceof ArithmeticException) { if (problem instanceof ArithmeticException) {
return future(new Callable<Integer>() { return future(new Callable<Integer>() {
public Integer call() { public Integer call() {
return 0; return 0;
} }
}, system.dispatcher()); }, ec);
} else } else
throw problem; throw problem;
} }
}); }, ec);
int result = Await.result(future, Duration.create(1, SECONDS)); int result = Await.result(future, Duration.create(1, SECONDS));
assertEquals(result, 0); assertEquals(result, 0);
//#try-recover //#try-recover
@ -408,6 +430,8 @@ public class FutureDocTestBase {
Future<String> future = Futures.successful("foo", system.dispatcher()); Future<String> future = Futures.successful("foo", system.dispatcher());
//#onSuccess //#onSuccess
final ExecutionContext ec = system.dispatcher();
future.onSuccess(new OnSuccess<String>() { future.onSuccess(new OnSuccess<String>() {
public void onSuccess(String result) { public void onSuccess(String result) {
if ("bar" == result) { if ("bar" == result) {
@ -416,12 +440,14 @@ public class FutureDocTestBase {
//Do something if it was some other String //Do something if it was some other String
} }
} }
}); }, ec);
//#onSuccess //#onSuccess
} }
{ {
Future<String> future = Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher()); Future<String> future = Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher());
//#onFailure //#onFailure
final ExecutionContext ec = system.dispatcher();
future.onFailure(new OnFailure() { future.onFailure(new OnFailure() {
public void onFailure(Throwable failure) { public void onFailure(Throwable failure) {
if (failure instanceof IllegalStateException) { if (failure instanceof IllegalStateException) {
@ -430,12 +456,14 @@ public class FutureDocTestBase {
//Do something if it was some other failure //Do something if it was some other failure
} }
} }
}); }, ec);
//#onFailure //#onFailure
} }
{ {
Future<String> future = Futures.successful("foo", system.dispatcher()); Future<String> future = Futures.successful("foo", system.dispatcher());
//#onComplete //#onComplete
final ExecutionContext ec = system.dispatcher();
future.onComplete(new OnComplete<String>() { future.onComplete(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) { public void onComplete(Throwable failure, String result) {
if (failure != null) { if (failure != null) {
@ -444,7 +472,7 @@ public class FutureDocTestBase {
// We got a result, do something with it // We got a result, do something with it
} }
} }
}); }, ec);
//#onComplete //#onComplete
} }
} }
@ -453,13 +481,14 @@ public class FutureDocTestBase {
public void useOrAndZip() throws Exception { public void useOrAndZip() throws Exception {
{ {
//#zip //#zip
Future<String> future1 = Futures.successful("foo", system.dispatcher()); final ExecutionContext ec = system.dispatcher();
Future<String> future2 = Futures.successful("bar", system.dispatcher()); Future<String> future1 = Futures.successful("foo", ec);
Future<String> future2 = Futures.successful("bar", ec);
Future<String> future3 = future1.zip(future2).map(new Mapper<scala.Tuple2<String, String>, String>() { Future<String> future3 = future1.zip(future2).map(new Mapper<scala.Tuple2<String, String>, String>() {
public String apply(scala.Tuple2<String, String> zipped) { public String apply(scala.Tuple2<String, String> zipped) {
return zipped._1() + " " + zipped._2(); return zipped._1() + " " + zipped._2();
} }
}); }, ec);
String result = Await.result(future3, Duration.create(1, SECONDS)); String result = Await.result(future3, Duration.create(1, SECONDS));
assertEquals("foo bar", result); assertEquals("foo bar", result);
@ -468,9 +497,10 @@ public class FutureDocTestBase {
{ {
//#fallback-to //#fallback-to
Future<String> future1 = Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher()); final ExecutionContext ec = system.dispatcher();
Future<String> future2 = Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher()); Future<String> future1 = Futures.failed(new IllegalStateException("OHNOES1"), ec);
Future<String> future3 = Futures.successful("bar", system.dispatcher()); Future<String> future2 = Futures.failed(new IllegalStateException("OHNOES2"), ec);
Future<String> future3 = Futures.successful("bar", ec);
Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case
String result = Await.result(future4, Duration.create(1, SECONDS)); String result = Await.result(future4, Duration.create(1, SECONDS));
assertEquals("bar", result); assertEquals("bar", result);

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 scala.concurrent.Await;
import scala.concurrent.Future;
import akka.testkit.CallingThreadDispatcher;
import akka.testkit.TestActor;
import akka.testkit.TestActor.AutoPilot;
import akka.testkit.TestActorRef;
import akka.testkit.JavaTestKit;
import scala.concurrent.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 scala.concurrent.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 * CallingThreadDispatcher
- This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads, - 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. for details and restrictions.
- Sharability: Unlimited - Sharability: Unlimited

View file

@ -4,11 +4,574 @@
Testing Actor Systems (Java) Testing Actor Systems (Java)
############################## ##############################
Due to the conciseness of test DSLs available for Scala, it may be a good idea As with any piece of software, automated tests are a very important part of the
to write the test suite in that language even if the main project is written in development cycle. The actor model presents a different view on how units of
Java. If that is not desirable, you can also use :class:`TestKit` and friends code are delimited and how they interact, which has an influence on how to
from Java, albeit with more verbose syntax Munish Gupta has `published a nice perform tests.
post <http://www.akkaessentials.in/2012/05/using-testkit-with-java.html>`_
showing several patterns you may find useful, and for reference documentation .. note::
please refer to :ref:`akka-testkit` until that section has been ported over to
cover Java in full. 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

@ -558,6 +558,56 @@ well. Use the ``getContext().unbecome`` method from within the Actor.
if (message.equals("revert")) getContext().unbecome(); if (message.equals("revert")) getContext().unbecome();
} }
Stash
=====
The ``UntypedActorWithStash`` class enables an actor to temporarily stash away messages
that can not or should not be handled using the actor's current
behavior. Upon changing the actor's message handler, i.e., right
before invoking ``getContext().become()`` or ``getContext().unbecome()``, all
stashed messages can be "unstashed", thereby prepending them to the actor's
mailbox. This way, the stashed messages can be processed in the same
order as they have been received originally.
.. warning::
Please note that the stash can only be used together with actors
that have a deque-based mailbox. For this, configure the
``mailbox-type`` of the dispatcher to be a deque-based mailbox, such as
``akka.dispatch.UnboundedDequeBasedMailbox`` (see :ref:`dispatchers-java`).
Here is an example of the ``UntypedActorWithStash`` class in action:
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#stash
Invoking ``stash()`` adds the current message (the message that the
actor received last) to the actor's stash. It is typically invoked
when handling the default case in the actor's message handler to stash
messages that aren't handled by the other cases. It is illegal to
stash the same message twice; to do so results in an
``IllegalStateException`` being thrown. The stash may also be bounded
in which case invoking ``stash()`` may lead to a capacity violation,
which results in a ``StashOverflowException``. The capacity of the
stash can be configured using the ``stash-capacity`` setting (an ``Int``) of the
dispatcher's configuration.
Invoking ``unstashAll()`` enqueues messages from the stash to the
actor's mailbox until the capacity of the mailbox (if any) has been
reached (note that messages from the stash are prepended to the
mailbox). In case a bounded mailbox overflows, a
``MessageQueueAppendFailedException`` is thrown.
The stash is guaranteed to be empty after calling ``unstashAll()``.
The stash is backed by a ``scala.collection.immutable.Vector``. As a
result, even a very large number of messages may be stashed without a
major impact on performance.
Note that the stash is not persisted across restarts of an actor,
unlike the actor's mailbox. Therefore, it should be managed like other
parts of the actor's state which have the same property.
Killing an Actor Killing an Actor
================ ================

View file

@ -620,6 +620,62 @@ Here's how you use the ``unbecome`` method:
} }
Stash
=====
The `Stash` trait enables an actor to temporarily stash away messages
that can not or should not be handled using the actor's current
behavior. Upon changing the actor's message handler, i.e., right
before invoking ``context.become`` or ``context.unbecome``, all
stashed messages can be "unstashed", thereby prepending them to the actor's
mailbox. This way, the stashed messages can be processed in the same
order as they have been received originally.
.. warning::
Please note that the ``Stash`` can only be used together with actors
that have a deque-based mailbox. For this, configure the
``mailbox-type`` of the dispatcher to be a deque-based mailbox, such as
``akka.dispatch.UnboundedDequeBasedMailbox`` (see :ref:`dispatchers-scala`).
Here is an example of the ``Stash`` in action:
.. includecode:: code/docs/actor/ActorDocSpec.scala#stash
Invoking ``stash()`` adds the current message (the message that the
actor received last) to the actor's stash. It is typically invoked
when handling the default case in the actor's message handler to stash
messages that aren't handled by the other cases. It is illegal to
stash the same message twice; to do so results in an
``IllegalStateException`` being thrown. The stash may also be bounded
in which case invoking ``stash()`` may lead to a capacity violation,
which results in a ``StashOverflowException``. The capacity of the
stash can be configured using the ``stash-capacity`` setting (an ``Int``) of the
dispatcher's configuration.
Invoking ``unstashAll()`` enqueues messages from the stash to the
actor's mailbox until the capacity of the mailbox (if any) has been
reached (note that messages from the stash are prepended to the
mailbox). In case a bounded mailbox overflows, a
``MessageQueueAppendFailedException`` is thrown.
The stash is guaranteed to be empty after calling ``unstashAll()``.
The stash is backed by a ``scala.collection.immutable.Vector``. As a
result, even a very large number of messages may be stashed without a
major impact on performance.
.. warning::
Note that the ``Stash`` trait must be mixed into (a subclass of) the
``Actor`` trait before any trait/class that overrides the ``preRestart``
callback. This means it's not possible to write
``Actor with MyActor with Stash`` if ``MyActor`` overrides ``preRestart``.
Note that the stash is not persisted across restarts of an actor,
unlike the actor's mailbox. Therefore, it should be managed like other
parts of the actor's state which have the same property.
Killing an Actor Killing an Actor
================ ================

View file

@ -302,6 +302,26 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
val actor = system.actorOf(Props(new HotSwapActor), name = "hot") val actor = system.actorOf(Props(new HotSwapActor), name = "hot")
} }
"using Stash" in {
//#stash
import akka.actor.Stash
class ActorWithProtocol extends Actor with Stash {
def receive = {
case "open"
unstashAll()
context.become {
case "write" // do writing...
case "close"
unstashAll()
context.unbecome()
case msg stash()
}
case msg stash()
}
}
//#stash
}
"using watch" in { "using watch" in {
//#watch //#watch
import akka.actor.{ Actor, Props, Terminated } import akka.actor.{ Actor, Props, Terminated }

View file

@ -6,8 +6,7 @@ package docs.actor
import language.postfixOps import language.postfixOps
//#imports //#imports
import akka.dispatch.{ Promise, Future } import scala.concurrent.{ Promise, Future, Await }
import scala.concurrent.Await
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import akka.actor.{ ActorContext, TypedActor, TypedProps } import akka.actor.{ ActorContext, TypedActor, TypedProps }
@ -40,7 +39,7 @@ class SquarerImpl(val name: String) extends Squarer {
def squareDontCare(i: Int): Unit = i * i //Nobody cares :( def squareDontCare(i: Int): Unit = i * i //Nobody cares :(
def square(i: Int): Future[Int] = Promise successful i * i def square(i: Int): Future[Int] = Promise.successful(i * i).future
def squareNowPlease(i: Int): Option[Int] = Some(i * i) def squareNowPlease(i: Int): Option[Int] = Some(i * i)
@ -56,7 +55,7 @@ trait Foo {
trait Bar { trait Bar {
import TypedActor.dispatcher //So we have an implicit dispatcher for our Promise import TypedActor.dispatcher //So we have an implicit dispatcher for our Promise
def doBar(str: String): Future[String] = Promise successful str.toUpperCase def doBar(str: String): Future[String] = Promise.successful(str.toUpperCase).future
} }
class FooBar extends Foo with Bar class FooBar extends Foo with Bar

View file

@ -5,17 +5,13 @@ package docs.future
import language.postfixOps import language.postfixOps
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
import org.scalatest.matchers.MustMatchers
import akka.testkit._ import akka.testkit._
import akka.actor.Actor import akka.actor.{ Actor, Props }
import akka.actor.Props
import akka.actor.Status.Failure import akka.actor.Status.Failure
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import java.lang.IllegalStateException import java.lang.IllegalStateException
import akka.dispatch.{ Future, Promise } import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
import scala.concurrent.{ Await, ExecutionContext }
object FutureDocSpec { object FutureDocSpec {
@ -45,7 +41,7 @@ class FutureDocSpec extends AkkaSpec {
"demonstrate usage custom ExecutionContext" in { "demonstrate usage custom ExecutionContext" in {
val yourExecutorServiceGoesHere = java.util.concurrent.Executors.newSingleThreadExecutor() val yourExecutorServiceGoesHere = java.util.concurrent.Executors.newSingleThreadExecutor()
//#diy-execution-context //#diy-execution-context
import akka.dispatch.{ ExecutionContext, Promise } import scala.concurrent.{ ExecutionContext, Promise }
implicit val ec = ExecutionContext.fromExecutorService(yourExecutorServiceGoesHere) implicit val ec = ExecutionContext.fromExecutorService(yourExecutorServiceGoesHere)
@ -120,7 +116,7 @@ class FutureDocSpec extends AkkaSpec {
val f1 = Future { val f1 = Future {
"Hello" + "World" "Hello" + "World"
} }
val f2 = Promise.successful(3) val f2 = Promise.successful(3).future
val f3 = f1 map { x val f3 = f1 map { x
f2 map { y f2 map { y
x.length * y x.length * y
@ -135,7 +131,7 @@ class FutureDocSpec extends AkkaSpec {
val f1 = Future { val f1 = Future {
"Hello" + "World" "Hello" + "World"
} }
val f2 = Promise.successful(3) val f2 = Promise.successful(3).future
val f3 = f1 flatMap { x val f3 = f1 flatMap { x
f2 map { y f2 map { y
x.length * y x.length * y
@ -148,7 +144,7 @@ class FutureDocSpec extends AkkaSpec {
"demonstrate usage of filter" in { "demonstrate usage of filter" in {
//#filter //#filter
val future1 = Promise.successful(4) val future1 = Promise.successful(4).future
val future2 = future1.filter(_ % 2 == 0) val future2 = future1.filter(_ % 2 == 0)
val result = Await.result(future2, 1 second) val result = Await.result(future2, 1 second)
result must be(4) result must be(4)
@ -293,8 +289,8 @@ class FutureDocSpec extends AkkaSpec {
val msg1 = -1 val msg1 = -1
//#try-recover //#try-recover
val future = akka.pattern.ask(actor, msg1) recoverWith { val future = akka.pattern.ask(actor, msg1) recoverWith {
case e: ArithmeticException Promise.successful(0) case e: ArithmeticException Promise.successful(0).future
case foo: IllegalArgumentException Promise.failed[Int](new IllegalStateException("All br0ken!")) case foo: IllegalArgumentException Promise.failed[Int](new IllegalStateException("All br0ken!")).future
} }
//#try-recover //#try-recover
Await.result(future, 1 second) must be(0) Await.result(future, 1 second) must be(0)
@ -346,7 +342,7 @@ class FutureDocSpec extends AkkaSpec {
Await.result(future, 1 second) must be("foo") Await.result(future, 1 second) must be("foo")
} }
{ {
val future = Promise.failed[String](new IllegalStateException("OHNOES")) val future = Promise.failed[String](new IllegalStateException("OHNOES")).future
//#onFailure //#onFailure
future onFailure { future onFailure {
case ise: IllegalStateException if ise.getMessage == "OHNOES" case ise: IllegalStateException if ise.getMessage == "OHNOES"
@ -372,10 +368,10 @@ class FutureDocSpec extends AkkaSpec {
"demonstrate usage of Promise.success & Promise.failed" in { "demonstrate usage of Promise.success & Promise.failed" in {
//#successful //#successful
val future = Promise.successful("Yay!") val future = Promise.successful("Yay!").future
//#successful //#successful
//#failed //#failed
val otherFuture = Promise.failed[String](new IllegalArgumentException("Bang!")) val otherFuture = Promise.failed[String](new IllegalArgumentException("Bang!")).future
//#failed //#failed
Await.result(future, 1 second) must be("Yay!") Await.result(future, 1 second) must be("Yay!")
intercept[IllegalArgumentException] { Await.result(otherFuture, 1 second) } intercept[IllegalArgumentException] { Await.result(otherFuture, 1 second) }

View file

@ -9,7 +9,7 @@ import language.postfixOps
import akka.testkit.TestProbe import akka.testkit.TestProbe
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import akka.actor._ import akka.actor._
import scala.concurrent.Futures import scala.concurrent.Future
//#imports-test-probe //#imports-test-probe
@ -127,7 +127,10 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val actorRef = TestActorRef(new MyActor) val actorRef = TestActorRef(new MyActor)
// hypothetical message stimulating a '42' answer // hypothetical message stimulating a '42' answer
val result = Await.result((actorRef ? Say42), 5 seconds).asInstanceOf[Int] val future = actorRef ? Say42
val result = future.value.get match {
case Right(x: Int) x
}
result must be(42) result must be(42)
//#test-behavior //#test-behavior
} }
@ -148,7 +151,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val actorRef = TestActorRef(new Actor { val actorRef = TestActorRef(new Actor {
def receive = { def receive = {
case boom throw new IllegalArgumentException("boom") case "hello" throw new IllegalArgumentException("boom")
} }
}) })
intercept[IllegalArgumentException] { actorRef.receive("hello") } intercept[IllegalArgumentException] { actorRef.receive("hello") }
@ -274,4 +277,15 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
//#test-kit-base //#test-kit-base
} }
"demonstrate within() nesting" in {
intercept[AssertionError] {
//#test-within-probe
val probe = TestProbe()
within(1 second) {
probe.expectMsg("hello")
}
//#test-within-probe
}
}
} }

View file

@ -93,7 +93,7 @@ There are 4 different types of message dispatchers:
* CallingThreadDispatcher * CallingThreadDispatcher
- This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads, - 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:`Scala-CallingThreadDispatcher`
for details and restrictions. for details and restrictions.
- Sharability: Unlimited - Sharability: Unlimited

View file

@ -424,7 +424,7 @@ This FSM will log at DEBUG level:
* all state transitions * all state transitions
Life cycle changes and special messages can be logged as described for Life cycle changes and special messages can be logged as described for
:ref:`Actors <actor.logging>`. :ref:`Actors <actor.logging-scala>`.
Rolling Event Log Rolling Event Log
----------------- -----------------

View file

@ -67,15 +67,6 @@ 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 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. any unit testing tool to bear on your actor as usual.
Expecting Exceptions
--------------------
Testing that an expected exception is thrown while processing a message sent to
the actor under test can be done by using a :class:`TestActorRef` :meth:`receive` based
invocation:
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions
.. _TestFSMRef: .. _TestFSMRef:
Testing Finite State Machines Testing Finite State Machines
@ -111,8 +102,8 @@ operation to complement the :class:`Actor` testing: it supports all operations
also valid on normal :class:`ActorRef`. Messages sent to the actor are 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 processed synchronously on the current thread and answers may be sent back as
usual. This trick is made possible by the :class:`CallingThreadDispatcher` usual. This trick is made possible by the :class:`CallingThreadDispatcher`
described below; this dispatcher is set implicitly for any actor instantiated described below (see `CallingThreadDispatcher`_); this dispatcher is set
into a :class:`TestActorRef`. implicitly for any actor instantiated into a :class:`TestActorRef`.
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-behavior .. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-behavior
@ -128,14 +119,14 @@ One more special aspect which is overridden for single-threaded tests is the
:meth:`receiveTimeout`, as including that would entail asynchronous queuing of :meth:`receiveTimeout`, as including that would entail asynchronous queuing of
:obj:`ReceiveTimeout` messages, violating the synchronous contract. :obj:`ReceiveTimeout` messages, violating the synchronous contract.
.. warning:: .. note::
To summarize: :class:`TestActorRef` overwrites two fields: it sets the To summarize: :class:`TestActorRef` overwrites two fields: it sets the
dispatcher to :obj:`CallingThreadDispatcher.global` and it sets the dispatcher to :obj:`CallingThreadDispatcher.global` and it sets the
:obj:`receiveTimeout` to None. :obj:`receiveTimeout` to None.
The Way In-Between The Way In-Between: Expecting Exceptions
------------------ ----------------------------------------
If you want to test the actor behavior, including hotswapping, but without If you want to test the actor behavior, including hotswapping, but without
involving a dispatcher and without having the :class:`TestActorRef` swallow involving a dispatcher and without having the :class:`TestActorRef` swallow
@ -143,10 +134,7 @@ 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 the :meth:`receive` method :class:`TestActorRef`, which will be forwarded to the
underlying actor: underlying actor:
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-unhandled .. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions
The above sample assumes the default behavior for unhandled messages, i.e.
that the actor doesn't swallow all messages and doesn't override :meth:`unhandled`.
Use Cases Use Cases
--------- ---------
@ -205,12 +193,12 @@ Built-In Assertions
The above mentioned :meth:`expectMsg` is not the only method for formulating The above mentioned :meth:`expectMsg` is not the only method for formulating
assertions concerning received messages. Here is the full list: assertions concerning received messages. Here is the full list:
* :meth:`expectMsg[T](d: Duration, msg: T): T` * :meth:`expectMsg[T](d: Duration, msg: T): T`
The given message object must be received within the specified time; the The given message object must be received within the specified time; the
object will be returned. object will be returned.
* :meth:`expectMsgPF[T](d: Duration)(pf: PartialFunction[Any, T]): T` * :meth:`expectMsgPF[T](d: Duration)(pf: PartialFunction[Any, T]): T`
Within the given time period, a message must be received and the given Within the given time period, a message must be received and the given
partial function must be defined for that message; the result from applying partial function must be defined for that message; the result from applying
@ -219,40 +207,40 @@ assertions concerning received messages. Here is the full list:
the deadline from the innermost enclosing :ref:`within <TestKit.within>` the deadline from the innermost enclosing :ref:`within <TestKit.within>`
block instead. block instead.
* :meth:`expectMsgClass[T](d: Duration, c: Class[T]): T` * :meth:`expectMsgClass[T](d: Duration, c: Class[T]): T`
An object which is an instance of the given :class:`Class` must be received 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 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, have a look at does a conformance check; if you need the class to be equal, have a look at
:meth:`expectMsgAllClassOf` with a single given class argument. :meth:`expectMsgAllClassOf` with a single given class argument.
* :meth:`expectMsgType[T: Manifest](d: Duration)` * :meth:`expectMsgType[T: Manifest](d: Duration)`
An object which is an instance of the given type (after erasure) must be An object which is an instance of the given type (after erasure) must be
received within the allotted time frame; the object will be returned. This received within the allotted time frame; the object will be returned. This
method is approximately equivalent to method is approximately equivalent to
``expectMsgClass(manifest[T].erasure)``. ``expectMsgClass(manifest[T].erasure)``.
* :meth:`expectMsgAnyOf[T](d: Duration, obj: T*): T` * :meth:`expectMsgAnyOf[T](d: Duration, obj: T*): T`
An object must be received within the given time, and it must be equal ( An object must be received within the given time, and it must be equal (
compared with ``==``) to at least one of the passed reference objects; the compared with ``==``) to at least one of the passed reference objects; the
received object will be returned. received object will be returned.
* :meth:`expectMsgAnyClassOf[T](d: Duration, obj: Class[_ <: T]*): T` * :meth:`expectMsgAnyClassOf[T](d: Duration, obj: Class[_ <: T]*): T`
An object must be received within the given time, and it must be an 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 instance of at least one of the supplied :class:`Class` objects; the
received object will be returned. received object will be returned.
* :meth:`expectMsgAllOf[T](d: Duration, obj: T*): Seq[T]` * :meth:`expectMsgAllOf[T](d: Duration, obj: T*): Seq[T]`
A number of objects matching the size of the supplied object array must be 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 received within the given time, and for each of the given objects there
must exist at least one among the received ones which equals (compared with must exist at least one among the received ones which equals (compared with
``==``) it. The full sequence of received objects is returned. ``==``) it. The full sequence of received objects is returned.
* :meth:`expectMsgAllClassOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]` * :meth:`expectMsgAllClassOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]`
A number of objects matching the size of the supplied :class:`Class` array A number of objects matching the size of the supplied :class:`Class` array
must be received within the given time, and for each of the given classes must be received within the given time, and for each of the given classes
@ -260,25 +248,25 @@ assertions concerning received messages. Here is the full list:
(compared with ``==``) it (this is *not* a conformance check). The full (compared with ``==``) it (this is *not* a conformance check). The full
sequence of received objects is returned. sequence of received objects is returned.
* :meth:`expectMsgAllConformingOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]` * :meth:`expectMsgAllConformingOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]`
A number of objects matching the size of the supplied :class:`Class` array A number of objects matching the size of the supplied :class:`Class` array
must be received within the given time, and for each of the given classes must be received within the given time, and for each of the given classes
there must exist at least one among the received objects which is an there must exist at least one among the received objects which is an
instance of this class. The full sequence of received objects is returned. instance of this class. The full sequence of received objects is returned.
* :meth:`expectNoMsg(d: Duration)` * :meth:`expectNoMsg(d: Duration)`
No message must be received within the given time. This also fails if a 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 message has been received before calling this method which has not been
removed from the queue using one of the other methods. removed from the queue using one of the other methods.
* :meth:`receiveN(n: Int, d: Duration): Seq[AnyRef]` * :meth:`receiveN(n: Int, d: Duration): Seq[AnyRef]`
``n`` messages must be received within the given time; the received ``n`` messages must be received within the given time; the received
messages are returned. messages are returned.
* :meth:`fishForMessage(max: Duration, hint: String)(pf: PartialFunction[Any, Boolean]): Any` * :meth:`fishForMessage(max: Duration, hint: String)(pf: PartialFunction[Any, Boolean]): Any`
Keep receiving messages as long as the time is not used up and the partial Keep receiving messages as long as the time is not used up and the partial
function matches and returns ``false``. Returns the message received for function matches and returns ``false``. Returns the message received for
@ -288,13 +276,13 @@ assertions concerning received messages. Here is the full list:
In addition to message reception assertions there are also methods which help In addition to message reception assertions there are also methods which help
with message flows: with message flows:
* :meth:`receiveOne(d: Duration): AnyRef` * :meth:`receiveOne(d: Duration): AnyRef`
Tries to receive one message for at most the given time interval and Tries to receive one message for at most the given time interval and
returns ``null`` in case of failure. If the given Duration is zero, the returns ``null`` in case of failure. If the given Duration is zero, the
call is non-blocking (polling mode). call is non-blocking (polling mode).
* :meth:`receiveWhile[T](max: Duration, idle: Duration, messages: Int)(pf: PartialFunction[Any, T]): Seq[T]` * :meth:`receiveWhile[T](max: Duration, idle: Duration, messages: Int)(pf: PartialFunction[Any, T]): Seq[T]`
Collect messages as long as Collect messages as long as
@ -309,14 +297,14 @@ with message flows:
idle timeout feature). The number of expected messages defaults to idle timeout feature). The number of expected messages defaults to
``Int.MaxValue``, which effectively disables this limit. ``Int.MaxValue``, which effectively disables this limit.
* :meth:`awaitCond(p: => Boolean, max: Duration, interval: Duration)` * :meth:`awaitCond(p: => Boolean, max: Duration, interval: Duration)`
Poll the given condition every :obj:`interval` until it returns ``true`` or Poll the given condition every :obj:`interval` until it returns ``true`` or
the :obj:`max` duration is used up. The interval defaults to 100 ms and the the :obj:`max` duration is used up. The interval defaults to 100 ms and the
maximum defaults to the time remaining in the innermost enclosing maximum defaults to the time remaining in the innermost enclosing
:ref:`within <TestKit.within>` block. :ref:`within <TestKit.within>` block.
* :meth:`ignoreMsg(pf: PartialFunction[AnyRef, Boolean])` * :meth:`ignoreMsg(pf: PartialFunction[AnyRef, Boolean])`
:meth:`ignoreNoMsg` :meth:`ignoreNoMsg`
@ -329,8 +317,8 @@ with message flows:
This feature is useful e.g. when testing a logging system, where you want This feature is useful e.g. when testing a logging system, where you want
to ignore regular messages and are only interested in your specific ones. to ignore regular messages and are only interested in your specific ones.
Expecting Exceptions Expecting Log Messages
-------------------- ----------------------
Since an integration test does not allow to the internal processing of the Since an integration test does not allow to the internal processing of the
participating actors, verifying expected exceptions cannot be done directly. participating actors, verifying expected exceptions cannot be done directly.
@ -341,6 +329,20 @@ exceptions:
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#event-filter .. includecode:: code/docs/testkit/TestkitDocSpec.scala#event-filter
If a number of occurrences is specific—as demonstrated above—then ``intercept``
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 passed-in block of code 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]
.. _TestKit.within: .. _TestKit.within:
Timing Assertions Timing Assertions
@ -363,7 +365,7 @@ The block given to :meth:`within` must complete after a :ref:`Duration` which
is between :obj:`min` and :obj:`max`, where the former defaults to zero. The 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 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 time is implicitly available within the block to all examination methods, if
you do not specify it, is is inherited from the innermost enclosing you do not specify it, it is inherited from the innermost enclosing
:meth:`within` block. :meth:`within` block.
It should be noted that if the last message-receiving assertion of the block is It should be noted that if the last message-receiving assertion of the block is
@ -473,8 +475,9 @@ B``, as long as a certain protocol is obeyed.
.. includecode:: ../../akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala#autopilot .. includecode:: ../../akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala#autopilot
The :meth:`run` method must return the auto-pilot for the next message, wrapped The :meth:`run` method must return the auto-pilot for the next message, which
in an :class:`Option`; setting it to :obj:`None` terminates the auto-pilot. may be :class:`KeepRunning` to retain the current one or :class:`NoAutoPilot`
to switch it off.
Caution about Timing Assertions Caution about Timing Assertions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
@ -483,23 +486,13 @@ 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 as counter-intuitive: you need to remember that the nicely scoped deadline as
described :ref:`above <TestKit.within>` is local to each probe. Hence, probes described :ref:`above <TestKit.within>` is local to each probe. Hence, probes
do not react to each other's deadlines or to the deadline set in an enclosing do not react to each other's deadlines or to the deadline set in an enclosing
:class:`TestKit` instance:: :class:`TestKit` instance:
class SomeTest extends TestKit(_system: ActorSystem) with ImplicitSender { .. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-within-probe
val probe = TestProbe() Here, the ``expectMsg`` call will use the default timeout.
within(100 millis) { .. _Scala-CallingThreadDispatcher:
probe.expectMsg("hallo") // Will hang forever!
}
}
This test will hang indefinitely, because the :meth:`expectMsg` call does not
see any deadline. Currently, the only option is to use ``probe.within`` in the
above code to make it work; later versions may include lexically scoped
deadlines using implicit arguments.
.. _TestCallingThreadDispatcherRef:
CallingThreadDispatcher CallingThreadDispatcher
======================= =======================
@ -598,7 +591,7 @@ has to offer:
exception stack traces exception stack traces
- Exclusion of certain classes of dead-lock scenarios - Exclusion of certain classes of dead-lock scenarios
.. _actor.logging: .. _actor.logging-scala:
Tracing Actor Invocations Tracing Actor Invocations
========================= =========================

View file

@ -5,7 +5,8 @@ package akka.osgi.aries.blueprint
import org.osgi.framework.BundleContext import org.osgi.framework.BundleContext
import akka.osgi.OsgiActorSystemFactory import akka.osgi.OsgiActorSystemFactory
import com.typesafe.config.ConfigFactory import akka.actor.ActorSystem
import com.typesafe.config.{ Config, ConfigFactory }
/** /**
* A set of helper/factory classes to build a Akka system using Blueprint. This class is only meant to be used by * A set of helper/factory classes to build a Akka system using Blueprint. This class is only meant to be used by
@ -18,30 +19,22 @@ class BlueprintActorSystemFactory(context: BundleContext, name: String) extends
var config: Option[String] = None var config: Option[String] = None
lazy val system = super.createActorSystem(stringToOption(name)) lazy val system: ActorSystem = super.createActorSystem(if (name == null || name.isEmpty) None else Some(name))
def setConfig(config: String) = { this.config = Some(config) } def setConfig(config: String): Unit = this.config = Some(config)
def create = system def create(): ActorSystem = system
def destroy = system.shutdown() def destroy(): Unit = system.shutdown()
def stringToOption(original: String) = if (original == null || original.isEmpty) {
None
} else {
Some(original)
}
/** /**
* Strategy method to create the Config for the ActorSystem, ensuring that the default/reference configuration is * Strategy method to create the Config for the ActorSystem, ensuring that the default/reference configuration is
* loaded from the akka-actor bundle. * loaded from the akka-actor bundle.
*/ */
override def actorSystemConfig(context: BundleContext) = { override def actorSystemConfig(context: BundleContext): Config =
config match { config match {
case Some(value) ConfigFactory.parseString(value).withFallback(super.actorSystemConfig(context)) case Some(value) ConfigFactory.parseString(value).withFallback(super.actorSystemConfig(context))
case None super.actorSystemConfig(context) case None super.actorSystemConfig(context)
} }
}
} }

View file

@ -1,18 +1,30 @@
package akka.osgi.aries.blueprint package akka.osgi.aries.blueprint
import org.apache.aries.blueprint.ParserContext
import org.osgi.service.blueprint.container.ComponentDefinitionException
import org.apache.aries.blueprint.mutable.MutableBeanMetadata
import collection.JavaConversions.setAsJavaSet
import org.osgi.framework.BundleContext import org.osgi.framework.BundleContext
import org.osgi.service.blueprint.container.ComponentDefinitionException
import org.osgi.service.blueprint.reflect.{ BeanMetadata, ComponentMetadata }
import org.apache.aries.blueprint.ParserContext
import org.apache.aries.blueprint.mutable.MutableBeanMetadata
import org.apache.aries.blueprint.reflect.{ ValueMetadataImpl, RefMetadataImpl, BeanArgumentImpl } import org.apache.aries.blueprint.reflect.{ ValueMetadataImpl, RefMetadataImpl, BeanArgumentImpl }
import org.w3c.dom.{ Element, Node } import org.w3c.dom.{ Element, Node }
import org.osgi.service.blueprint.reflect.{ BeanMetadata, ComponentMetadata }
import akka.actor.ActorSystem
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import ParserHelper.childElements import akka.actor.ActorSystem
import scala.annotation.tailrec
import java.net.URL
object NamespaceHandler {
private val ID_ATTRIBUTE = "id"
private val NAME_ATTRIBUTE = "name"
private val BUNDLE_CONTEXT_REFID = "blueprintBundleContext"
private val ACTORSYSTEM_ELEMENT_NAME = "actor-system"
private val CONFIG_ELEMENT_NAME = "config"
private val DESTROY_METHOD_NAME = "destroy"
private val FACTORY_METHOD_NAME = "create"
}
/** /**
* Aries Blueprint namespace handler implementation. This namespace handler will allow users of Apache Aries' Blueprint * Aries Blueprint namespace handler implementation. This namespace handler will allow users of Apache Aries' Blueprint
@ -40,18 +52,17 @@ class NamespaceHandler extends org.apache.aries.blueprint.NamespaceHandler {
import NamespaceHandler._ import NamespaceHandler._
val idCounter = new AtomicInteger(0) protected val idCounter = new AtomicInteger(0)
def getSchemaLocation(namespace: String) = getClass().getResource("akka.xsd") override def getSchemaLocation(namespace: String): URL = getClass().getResource("akka.xsd")
def getManagedClasses = setAsJavaSet(Set(classOf[BlueprintActorSystemFactory])) override def getManagedClasses = java.util.Collections.singleton(classOf[BlueprintActorSystemFactory])
def parse(element: Element, context: ParserContext) = element.getLocalName match { override def parse(element: Element, context: ParserContext) =
case ACTORSYSTEM_ELEMENT_NAME parseActorSystem(element, context) if (element.getLocalName == ACTORSYSTEM_ELEMENT_NAME) parseActorSystem(element, context)
case _ throw new ComponentDefinitionException("Unexpected element for Akka namespace: %s".format(element)) else throw new ComponentDefinitionException("Unexpected element for Akka namespace: %s".format(element))
}
def decorate(node: Node, component: ComponentMetadata, context: ParserContext) = override def decorate(node: Node, component: ComponentMetadata, context: ParserContext) =
throw new ComponentDefinitionException("Bad xml syntax: node decoration is not supported") throw new ComponentDefinitionException("Bad xml syntax: node decoration is not supported")
/* /*
@ -60,11 +71,12 @@ class NamespaceHandler extends org.apache.aries.blueprint.NamespaceHandler {
def parseActorSystem(element: Element, context: ParserContext) = { def parseActorSystem(element: Element, context: ParserContext) = {
val factory = createFactoryBean(context, element.getAttribute(NAME_ATTRIBUTE)) val factory = createFactoryBean(context, element.getAttribute(NAME_ATTRIBUTE))
for (child childElements(element)) { val nodelist = element.getChildNodes
child.getLocalName match { (0 until nodelist.getLength) collect {
case CONFIG_ELEMENT_NAME parseConfig(child, context, factory) case idx if nodelist.item(idx).getNodeType == Node.ELEMENT_NODE nodelist.item(idx).asInstanceOf[Element]
case _ throw new ComponentDefinitionException("Unexpected child element %s found in %s".format(child, element)) } foreach {
} case child if child.getLocalName == CONFIG_ELEMENT_NAME parseConfig(child, context, factory)
case child throw new ComponentDefinitionException("Unexpected child element %s found in %s".format(child, element))
} }
createActorSystemBean(context, element, factory) createActorSystemBean(context, element, factory)
@ -73,16 +85,23 @@ class NamespaceHandler extends org.apache.aries.blueprint.NamespaceHandler {
/* /*
* Parse <akka:config/> * Parse <akka:config/>
*/ */
def parseConfig(node: Element, context: ParserContext, factory: MutableBeanMetadata) = { protected def parseConfig(node: Element, context: ParserContext, factory: MutableBeanMetadata) =
factory.addProperty("config", new ValueMetadataImpl(node.getTextContent)) factory.addProperty(CONFIG_ELEMENT_NAME, new ValueMetadataImpl(node.getTextContent))
}
@tailrec protected final def findAvailableId(context: ParserContext): String =
".akka-" + idCounter.incrementAndGet() match {
case id if context.getComponentDefinitionRegistry.containsComponentDefinition(id) findAvailableId(context)
case available available
}
/* /*
* Create the bean definition for the ActorSystem * Create the bean definition for the ActorSystem
*/ */
def createActorSystemBean(context: ParserContext, element: Element, factory: MutableBeanMetadata): MutableBeanMetadata = { protected def createActorSystemBean(context: ParserContext, element: Element, factory: MutableBeanMetadata): MutableBeanMetadata = {
val system = context.createMetadata(classOf[MutableBeanMetadata]) val system = context.createMetadata(classOf[MutableBeanMetadata])
system.setId(getId(context, element)) val id = if (element.hasAttribute(ID_ATTRIBUTE)) element.getAttribute(ID_ATTRIBUTE) else findAvailableId(context)
system.setId(id)
system.setFactoryComponent(factory) system.setFactoryComponent(factory)
system.setFactoryMethod(FACTORY_METHOD_NAME) system.setFactoryMethod(FACTORY_METHOD_NAME)
@ -93,7 +112,7 @@ class NamespaceHandler extends org.apache.aries.blueprint.NamespaceHandler {
/* /*
* Create the bean definition for the BlueprintActorSystemFactory * Create the bean definition for the BlueprintActorSystemFactory
*/ */
def createFactoryBean(context: ParserContext, name: String): MutableBeanMetadata = { protected def createFactoryBean(context: ParserContext, name: String): MutableBeanMetadata = {
val factory = context.createMetadata(classOf[MutableBeanMetadata]) val factory = context.createMetadata(classOf[MutableBeanMetadata])
factory.setId(findAvailableId(context)) factory.setId(findAvailableId(context))
factory.setScope(BeanMetadata.SCOPE_SINGLETON) factory.setScope(BeanMetadata.SCOPE_SINGLETON)
@ -108,41 +127,4 @@ class NamespaceHandler extends org.apache.aries.blueprint.NamespaceHandler {
context.getComponentDefinitionRegistry.registerComponentDefinition(factory) context.getComponentDefinitionRegistry.registerComponentDefinition(factory)
factory factory
} }
}
/*
* Get the assigned id or generate a suitable id
*/
def getId(context: ParserContext, element: Element) = {
if (element.hasAttribute(ID_ATTRIBUTE)) {
element.getAttribute(ID_ATTRIBUTE)
} else {
findAvailableId(context)
}
}
/*
* Find the next available component id
*/
def findAvailableId(context: ParserContext): String = {
val id = ".akka-" + idCounter.incrementAndGet()
if (context.getComponentDefinitionRegistry.containsComponentDefinition(id)) {
// id already exists, let's try the next one
findAvailableId(context)
} else id
}
}
object NamespaceHandler {
private val ID_ATTRIBUTE = "id"
private val NAME_ATTRIBUTE = "name"
private val BUNDLE_CONTEXT_REFID = "blueprintBundleContext"
private val ACTORSYSTEM_ELEMENT_NAME = "actor-system"
private val CONFIG_ELEMENT_NAME = "config"
private val DESTROY_METHOD_NAME = "destroy"
private val FACTORY_METHOD_NAME = "create"
}

View file

@ -1,20 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.osgi.aries.blueprint
import org.w3c.dom.{ Node, Element }
/**
* Helper class to deal with the W3C DOM types
*/
object ParserHelper {
def childElements(element: Element): Seq[Element] =
children(element).filter(_.getNodeType == Node.ELEMENT_NODE).asInstanceOf[Seq[Element]]
private[this] def children(element: Element): Seq[Node] = {
val nodelist = element.getChildNodes
for (index 0 until nodelist.getLength) yield nodelist.item(index)
}
}

View file

@ -55,10 +55,13 @@ abstract class ActorSystemActivator extends BundleActivator {
* Register the actor system in the OSGi service registry. The activator itself will ensure that this service * Register the actor system in the OSGi service registry. The activator itself will ensure that this service
* is unregistered again when the bundle is being stopped. * is unregistered again when the bundle is being stopped.
* *
* Only one ActorSystem can be registered at a time, so any previous registration will be unregistered prior to registering the new.
*
* @param context the bundle context * @param context the bundle context
* @param system the actor system * @param system the actor system
*/ */
def registerService(context: BundleContext, system: ActorSystem): Unit = { def registerService(context: BundleContext, system: ActorSystem): Unit = {
registration.foreach(_.unregister()) //Cleanup
val properties = new Properties() val properties = new Properties()
properties.put("name", system.name) properties.put("name", system.name)
registration = Some(context.registerService(classOf[ActorSystem].getName, system, registration = Some(context.registerService(classOf[ActorSystem].getName, system,

View file

@ -17,7 +17,7 @@ class OsgiActorSystemFactory(val context: BundleContext) {
/* /*
* Classloader that delegates to the bundle for which the factory is creating an ActorSystem * Classloader that delegates to the bundle for which the factory is creating an ActorSystem
*/ */
private val classloader = BundleDelegatingClassLoader.createFor(context) private val classloader = BundleDelegatingClassLoader(context)
/** /**
* Creates the [[akka.actor.ActorSystem]], using the name specified * Creates the [[akka.actor.ActorSystem]], using the name specified
@ -36,25 +36,21 @@ class OsgiActorSystemFactory(val context: BundleContext) {
* Strategy method to create the Config for the ActorSystem, ensuring that the default/reference configuration is * Strategy method to create the Config for the ActorSystem, ensuring that the default/reference configuration is
* loaded from the akka-actor bundle. * loaded from the akka-actor bundle.
*/ */
def actorSystemConfig(context: BundleContext): Config = { def actorSystemConfig(context: BundleContext): Config =
val reference = ConfigFactory.defaultReference(classOf[ActorSystem].getClassLoader) ConfigFactory.load(classloader).withFallback(ConfigFactory.defaultReference(classOf[ActorSystem].getClassLoader))
ConfigFactory.load(classloader).withFallback(reference)
}
/** /**
* Determine the name for the [[akka.actor.ActorSystem]] * Determine the name for the [[akka.actor.ActorSystem]]
* Returns a default value of `bundle-<bundle id>-ActorSystem` is no name is being specified * Returns a default value of `bundle-<bundle id>-ActorSystem` is no name is being specified
*/ */
def actorSystemName(name: Option[String]): String = def actorSystemName(name: Option[String]): String =
name.getOrElse("bundle-%s-ActorSystem".format(context.getBundle().getBundleId)) name.getOrElse("bundle-%s-ActorSystem".format(context.getBundle.getBundleId))
} }
object OsgiActorSystemFactory { object OsgiActorSystemFactory {
/* /*
* Create an [[OsgiActorSystemFactory]] instance to set up Akka in an OSGi environment * Create an [[OsgiActorSystemFactory]] instance to set up Akka in an OSGi environment
*/ */
def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context) def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context)
} }

View file

@ -3,9 +3,10 @@
*/ */
package akka.osgi.impl package akka.osgi.impl
import language.existentials
import java.net.URL import java.net.URL
import java.util.Enumeration import java.util.Enumeration
import org.osgi.framework.{ BundleContext, Bundle } import org.osgi.framework.{ BundleContext, Bundle }
/* /*
@ -16,7 +17,7 @@ object BundleDelegatingClassLoader {
/* /*
* Create a bundle delegating classloader for the bundle context's bundle * Create a bundle delegating classloader for the bundle context's bundle
*/ */
def createFor(context: BundleContext) = new BundleDelegatingClassLoader(context.getBundle) def apply(context: BundleContext): BundleDelegatingClassLoader = new BundleDelegatingClassLoader(context.getBundle)
} }
@ -30,13 +31,11 @@ class BundleDelegatingClassLoader(bundle: Bundle, classLoader: Option[ClassLoade
protected override def findClass(name: String): Class[_] = bundle.loadClass(name) protected override def findClass(name: String): Class[_] = bundle.loadClass(name)
protected override def findResource(name: String): URL = { protected override def findResource(name: String): URL =
val resource = bundle.getResource(name) bundle.getResource(name) match {
classLoader match { case null if classLoader.isDefined classLoader.get.getResource(name)
case Some(loader) if resource == null loader.getResource(name) case result result
case _ resource
} }
}
@SuppressWarnings(Array("unchecked", "rawtypes")) @SuppressWarnings(Array("unchecked", "rawtypes"))
protected override def findResources(name: String): Enumeration[URL] = protected override def findResources(name: String): Enumeration[URL] =
@ -44,32 +43,18 @@ class BundleDelegatingClassLoader(bundle: Bundle, classLoader: Option[ClassLoade
protected override def loadClass(name: String, resolve: Boolean): Class[_] = { protected override def loadClass(name: String, resolve: Boolean): Class[_] = {
val clazz = try { val clazz = try {
findClass(name) try findClass(name) catch { case _: ClassNotFoundException if classLoader.isDefined classLoader.get.loadClass(name) } // First fall back to secondary loader
} catch { } catch {
case cnfe: ClassNotFoundException { case cnfe: ClassNotFoundException
classLoader match { throw new ClassNotFoundException("%s from bundle %s (%s)".format(name, bundle.getBundleId, bundle.getSymbolicName), cnfe) // IF we have no secondary loader or that failed as well, wrap and rethrow
case Some(loader) loadClass(name, loader)
case None rethrowClassNotFoundException(name, cnfe)
}
}
} }
if (resolve) {
if (resolve)
resolveClass(clazz) resolveClass(clazz)
}
clazz clazz
} }
private def loadClass(name: String, classLoader: ClassLoader) = override val toString: String = "BundleDelegatingClassLoader(%s)".format(bundle.getBundleId)
try {
classLoader.loadClass(name)
} catch {
case cnfe: ClassNotFoundException rethrowClassNotFoundException(name, cnfe)
}
private def rethrowClassNotFoundException(name: String, cnfe: ClassNotFoundException): Nothing =
throw new ClassNotFoundException(name + " from bundle " + bundle.getBundleId + " (" + bundle.getSymbolicName + ")", cnfe)
override def toString: String = String.format("BundleDelegatingClassLoader(%s)", bundle)
} }

View file

@ -6,10 +6,9 @@ package akka.remote
import akka.actor._ import akka.actor._
import akka.dispatch._ import akka.dispatch._
import akka.event.{ Logging, LoggingAdapter } import akka.event.{ Logging, LoggingAdapter, EventStream }
import akka.event.EventStream import akka.serialization.{ Serialization, SerializationExtension }
import akka.serialization.Serialization import scala.concurrent.Future
import akka.serialization.SerializationExtension
/** /**
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
@ -38,7 +37,7 @@ private[akka] class RemoteActorRefProvider(
override def rootGuardian: InternalActorRef = local.rootGuardian override def rootGuardian: InternalActorRef = local.rootGuardian
override def guardian: LocalActorRef = local.guardian override def guardian: LocalActorRef = local.guardian
override def systemGuardian: LocalActorRef = local.systemGuardian override def systemGuardian: LocalActorRef = local.systemGuardian
override def terminationFuture: Promise[Unit] = local.terminationFuture override def terminationFuture: Future[Unit] = local.terminationFuture
override def dispatcher: MessageDispatcher = local.dispatcher override def dispatcher: MessageDispatcher = local.dispatcher
override def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = local.registerTempActor(actorRef, path) override def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = local.registerTempActor(actorRef, path)
override def unregisterTempActor(path: ActorPath): Unit = local.unregisterTempActor(path) override def unregisterTempActor(path: ActorPath): Unit = local.unregisterTempActor(path)

View file

@ -0,0 +1,329 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.testkit;
import scala.runtime.AbstractFunction0;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.event.Logging;
import akka.event.Logging.LogEvent;
import akka.japi.PurePartialFunction;
import akka.japi.CachingPartialFunction;
import akka.japi.Util;
import scala.concurrent.util.Duration;
/**
* Java API for the TestProbe. Proper JavaDocs to come once JavaDoccing is implemented.
*/
public class JavaTestKit {
private final TestProbe p;
public JavaTestKit(ActorSystem system) {
p = new TestProbe(system);
}
public ActorRef getRef() {
return p.ref();
}
public ActorSystem getSystem() {
return p.system();
}
static public Duration duration(String s) {
return Duration.parse(s);
}
public Duration dilated(Duration d) {
return d.mul(TestKitExtension.get(p.system()).TestTimeFactor());
}
public boolean msgAvailable() {
return p.msgAvailable();
}
public ActorRef getLastSender() {
return p.lastMessage().sender();
}
public void send(ActorRef actor, Object msg) {
actor.tell(msg, p.ref());
}
public void forward(ActorRef actor) {
actor.tell(p.lastMessage().msg(), p.lastMessage().sender());
}
public void reply(Object msg) {
p.lastMessage().sender().tell(msg, p.ref());
}
public Duration getRemainingTime() {
return p.remaining();
}
public Duration getRemainingTimeOr(Duration def) {
return p.remainingOr(def);
}
public ActorRef watch(ActorRef ref) {
return p.watch(ref);
}
public ActorRef unwatch(ActorRef ref) {
return p.unwatch(ref);
}
public abstract class IgnoreMsg {
abstract protected boolean ignore(Object msg);
public IgnoreMsg() {
p.ignoreMsg(new PurePartialFunction<Object, Object>() {
public Boolean apply(Object in, boolean isCheck) {
return ignore(in);
}
});
}
}
public void ignoreNoMsg() {
p.ignoreNoMsg();
}
public void setAutoPilot(TestActor.AutoPilot pilot) {
p.setAutoPilot(pilot);
}
public abstract class Within {
protected abstract void run();
public Within(Duration max) {
p.within(max, new AbstractFunction0<Object>() {
public Object apply() {
run();
return null;
}
});
}
public Within(Duration min, Duration max) {
p.within(min, max, new AbstractFunction0<Object>() {
public Object apply() {
run();
return null;
}
});
}
}
public abstract class AwaitCond {
protected abstract boolean cond();
public AwaitCond() {
this(Duration.Undefined(), p.awaitCond$default$3());
}
public AwaitCond(Duration max) {
this(max, p.awaitCond$default$3());
}
public AwaitCond(Duration max, Duration interval) {
p.awaitCond(new AbstractFunction0<Object>() {
public Object apply() {
return cond();
}
}, max, interval);
}
}
public abstract class ExpectMsg<T> {
private final T result;
public ExpectMsg(String hint) {
this(Duration.Undefined(), hint);
}
public ExpectMsg(Duration max, String hint) {
final Object received = p.receiveOne(max);
try {
result = match(received);
} catch (PurePartialFunction.NoMatchException ex) {
throw new AssertionError("while expecting '" + hint
+ "' received unexpected: " + received);
}
}
abstract protected T match(Object msg);
protected RuntimeException noMatch() {
throw PurePartialFunction.noMatch();
}
public T get() {
return result;
}
}
public <T> T expectMsgEquals(T msg) {
return p.expectMsg(msg);
}
public <T> T expectMsgEquals(Duration max, T msg) {
return p.expectMsg(max, msg);
}
public <T> T expectMsgClass(Class<T> clazz) {
return p.expectMsgClass(clazz);
}
public <T> T expectMsgClass(Duration max, Class<T> clazz) {
return p.expectMsgClass(max, clazz);
}
public Object expectMsgAnyOf(Object... msgs) {
return p.expectMsgAnyOf(Util.arrayToSeq(msgs));
}
public Object expectMsgAnyOf(Duration max, Object... msgs) {
return p.expectMsgAnyOf(max, Util.arrayToSeq(msgs));
}
public Object[] expectMsgAllOf(Object... msgs) {
return (Object[]) p.expectMsgAllOf(Util.arrayToSeq(msgs)).toArray(
Util.classTag(Object.class));
}
public Object[] expectMsgAllOf(Duration max, Object... msgs) {
return (Object[]) p.expectMsgAllOf(max, Util.arrayToSeq(msgs)).toArray(
Util.classTag(Object.class));
}
@SuppressWarnings("unchecked")
public <T> T expectMsgAnyClassOf(Class<? extends T>... classes) {
final Object result = p.expectMsgAnyClassOf(Util.arrayToSeq(classes));
return (T) result;
}
public Object expectMsgAnyClassOf(Duration max, Class<?>... classes) {
return p.expectMsgAnyClassOf(max, Util.arrayToSeq(classes));
}
public void expectNoMsg() {
p.expectNoMsg();
}
public void expectNoMsg(Duration max) {
p.expectNoMsg(max);
}
public abstract class ReceiveWhile<T> {
abstract protected T match(Object msg);
private Object results;
public ReceiveWhile(Class<T> clazz) {
this(clazz, Duration.Undefined());
}
public ReceiveWhile(Class<T> clazz, Duration max) {
this(clazz, max, Duration.Inf(), Integer.MAX_VALUE);
}
public ReceiveWhile(Class<T> clazz, Duration max, int messages) {
this(clazz, max, Duration.Inf(), messages);
}
@SuppressWarnings("unchecked")
public ReceiveWhile(Class<T> clazz, Duration max, Duration idle, int messages) {
results = p.receiveWhile(max, idle, messages,
new CachingPartialFunction<Object, T>() {
public T match(Object msg) {
return ReceiveWhile.this.match(msg);
}
}).toArray(Util.classTag(clazz));
}
protected RuntimeException noMatch() {
throw PurePartialFunction.noMatch();
}
@SuppressWarnings("unchecked")
public T[] get() {
return (T[]) results;
}
}
public abstract class EventFilter<T> {
abstract protected T run();
private final Class<? extends Logging.LogEvent> clazz;
private String source = null;
private String message = null;
private boolean pattern = false;
private boolean complete = false;
private int occurrences = Integer.MAX_VALUE;
private Class<? extends Throwable> exceptionType = null;
@SuppressWarnings("unchecked")
public EventFilter(Class<?> clazz) {
if (Throwable.class.isAssignableFrom(clazz)) {
this.clazz = Logging.Error.class;
exceptionType = (Class<? extends Throwable>) clazz;
} else if (Logging.LogEvent.class.isAssignableFrom(clazz)) {
this.clazz = (Class<? extends LogEvent>) clazz;
} else throw new IllegalArgumentException("supplied class must either be LogEvent or Throwable");
}
public T exec() {
akka.testkit.EventFilter filter;
if (clazz == Logging.Error.class) {
if (exceptionType == null) exceptionType = Logging.noCause().getClass();
filter = new ErrorFilter(exceptionType, source, message, pattern, complete, occurrences);
} else if (clazz == Logging.Warning.class) {
filter = new WarningFilter(source, message, pattern, complete, occurrences);
} else if (clazz == Logging.Info.class) {
filter = new InfoFilter(source, message, pattern, complete, occurrences);
} else if (clazz == Logging.Debug.class) {
filter = new DebugFilter(source, message, pattern, complete, occurrences);
} else throw new IllegalArgumentException("unknown LogLevel " + clazz);
return filter.intercept(new AbstractFunction0<T>() {
public T apply() {
return run();
}
}, p.system());
}
public EventFilter<T> message(String msg) {
message = msg;
pattern = false;
complete = true;
return this;
}
public EventFilter<T> startsWith(String msg) {
message = msg;
pattern = false;
complete = false;
return this;
}
public EventFilter<T> matches(String regex) {
message = regex;
pattern = true;
return this;
}
public EventFilter<T> from(String source) {
this.source = source;
return this;
}
public EventFilter<T> occurrences(int number) {
occurrences = number;
return this;
}
}
}

View file

@ -133,4 +133,9 @@ object TestActorRef {
"\nOR try to change: 'actorOf(Props[MyActor]' to 'actorOf(Props(new MyActor)'.", exception) "\nOR try to change: 'actorOf(Props[MyActor]' to 'actorOf(Props(new MyActor)'.", exception)
} }
}), name) }), name)
/**
* Java API
*/
def create[T <: Actor](system: ActorSystem, props: Props, name: String): TestActorRef[T] = apply(props, name)(system)
} }

View file

@ -13,12 +13,24 @@ import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atom
import atomic.AtomicInteger import atomic.AtomicInteger
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.util.{ Timeout, BoxedType } import akka.util.{ Timeout, BoxedType }
import scala.annotation.varargs
import akka.japi.PurePartialFunction
object TestActor { object TestActor {
type Ignore = Option[PartialFunction[AnyRef, Boolean]] type Ignore = Option[PartialFunction[AnyRef, Boolean]]
trait AutoPilot { abstract class AutoPilot {
def run(sender: ActorRef, msg: Any): Option[AutoPilot] def run(sender: ActorRef, msg: Any): AutoPilot
def noAutoPilot: AutoPilot = NoAutoPilot
def keepRunning: AutoPilot = KeepRunning
}
case object NoAutoPilot extends AutoPilot {
def run(sender: ActorRef, msg: Any): AutoPilot = this
}
case object KeepRunning extends AutoPilot {
def run(sender: ActorRef, msg: Any): AutoPilot = sys.error("must not call")
} }
case class SetIgnore(i: Ignore) case class SetIgnore(i: Ignore)
@ -42,15 +54,18 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
var ignore: Ignore = None var ignore: Ignore = None
var autopilot: Option[AutoPilot] = None var autopilot: AutoPilot = NoAutoPilot
def receive = { def receive = {
case SetIgnore(ign) ignore = ign case SetIgnore(ign) ignore = ign
case x @ Watch(ref) context.watch(ref); queue.offerLast(RealMessage(x, self)) case x @ Watch(ref) context.watch(ref); queue.offerLast(RealMessage(x, self))
case x @ UnWatch(ref) context.unwatch(ref); queue.offerLast(RealMessage(x, self)) case x @ UnWatch(ref) context.unwatch(ref); queue.offerLast(RealMessage(x, self))
case SetAutoPilot(pilot) autopilot = Some(pilot) case SetAutoPilot(pilot) autopilot = pilot
case x: AnyRef case x: AnyRef
autopilot = autopilot.flatMap(_.run(sender, x)) autopilot = autopilot.run(sender, x) match {
case KeepRunning autopilot
case other other
}
val observe = ignore map (ignoreFunc if (ignoreFunc isDefinedAt x) !ignoreFunc(x) else true) getOrElse true val observe = ignore map (ignoreFunc if (ignoreFunc isDefinedAt x) !ignoreFunc(x) else true) getOrElse true
if (observe) queue.offerLast(RealMessage(x, sender)) if (observe) queue.offerLast(RealMessage(x, sender))
} }
@ -130,20 +145,20 @@ trait TestKitBase {
* Have the testActor watch someone (i.e. `context.watch(...)`). Waits until * Have the testActor watch someone (i.e. `context.watch(...)`). Waits until
* the Watch message is received back using expectMsg. * the Watch message is received back using expectMsg.
*/ */
def watch(ref: ActorRef) { def watch(ref: ActorRef): ActorRef = {
val msg = TestActor.Watch(ref) val msg = TestActor.Watch(ref)
testActor ! msg testActor ! msg
expectMsg(msg) expectMsg(msg).ref
} }
/** /**
* Have the testActor stop watching someone (i.e. `context.unwatch(...)`). Waits until * Have the testActor stop watching someone (i.e. `context.unwatch(...)`). Waits until
* the Watch message is received back using expectMsg. * the Watch message is received back using expectMsg.
*/ */
def unwatch(ref: ActorRef) { def unwatch(ref: ActorRef): ActorRef = {
val msg = TestActor.UnWatch(ref) val msg = TestActor.UnWatch(ref)
testActor ! msg testActor ! msg
expectMsg(msg) expectMsg(msg).ref
} }
/** /**

View file

@ -4,13 +4,14 @@
package akka.testkit; package akka.testkit;
import org.junit.Test; import akka.actor.Actor;
import akka.actor.Props; import akka.actor.Props;
public class TestActorRefJavaCompile { public class TestActorRefJavaCompile {
public void shouldBeAbleToCompileWhenUsingApply() { public void shouldBeAbleToCompileWhenUsingApply() {
//Just a dummy call to make sure it compiles //Just a dummy call to make sure it compiles
TestActorRef ref = TestActorRef.apply(new Props(), null); TestActorRef<Actor> ref = TestActorRef.apply(new Props(), null);
ref.toString();
} }
} }

View file

@ -46,10 +46,10 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout {
//#autopilot //#autopilot
val probe = TestProbe() val probe = TestProbe()
probe.setAutoPilot(new TestActor.AutoPilot { probe.setAutoPilot(new TestActor.AutoPilot {
def run(sender: ActorRef, msg: Any): Option[TestActor.AutoPilot] = def run(sender: ActorRef, msg: Any): TestActor.AutoPilot =
msg match { msg match {
case "stop" None case "stop" TestActor.NoAutoPilot
case x testActor.tell(x, sender); Some(this) case x testActor.tell(x, sender); TestActor.KeepRunning
} }
}) })
//#autopilot //#autopilot

View file

@ -6,13 +6,13 @@ package akka.zeromq
import org.zeromq.ZMQ.{ Socket, Poller } import org.zeromq.ZMQ.{ Socket, Poller }
import org.zeromq.{ ZMQ JZMQ } import org.zeromq.{ ZMQ JZMQ }
import akka.actor._ import akka.actor._
import akka.dispatch.{ Promise, Future } import scala.concurrent.{ Promise, Future }
import akka.event.Logging
import scala.annotation.tailrec
import java.util.concurrent.TimeUnit
import collection.mutable.ListBuffer
import akka.util.NonFatal
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import scala.annotation.tailrec
import scala.collection.mutable.ListBuffer
import akka.event.Logging
import java.util.concurrent.TimeUnit
import akka.util.NonFatal
private[zeromq] object ConcurrentSocketActor { private[zeromq] object ConcurrentSocketActor {
private sealed trait PollMsg private sealed trait PollMsg
@ -173,12 +173,15 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A
// this is a PollMsg=>Unit which either polls or schedules Poll, depending on the sign of the timeout // this is a PollMsg=>Unit which either polls or schedules Poll, depending on the sign of the timeout
private val doPollTimeout = { private val doPollTimeout = {
val ext = ZeroMQExtension(context.system)
val fromConfig = params collectFirst { case PollTimeoutDuration(duration) duration } val fromConfig = params collectFirst { case PollTimeoutDuration(duration) duration }
val duration = fromConfig getOrElse ZeroMQExtension(context.system).DefaultPollTimeout val duration = (fromConfig getOrElse ext.DefaultPollTimeout)
if (duration > Duration.Zero) { (msg: PollMsg) if (duration > Duration.Zero) {
// for positive timeout values, do poll (i.e. block this thread) // for positive timeout values, do poll (i.e. block this thread)
poller.poll(duration.toMicros) val pollLength = duration.toUnit(ext.pollTimeUnit).toLong
self ! msg (msg: PollMsg)
poller.poll(pollLength)
self ! msg
} else { } else {
val d = -duration val d = -duration

View file

@ -48,7 +48,7 @@ sealed trait SocketOptionQuery extends Request
/** /**
* This socket should be a client socket and connect to the specified endpoint * This socket should be a client socket and connect to the specified endpoint
* *
* @param endpoint an uri like tcp://127.0.0.1.5432 * @param endpoint URI (ex. tcp://127.0.0.1:5432)
*/ */
case class Connect(endpoint: String) extends SocketConnectOption case class Connect(endpoint: String) extends SocketConnectOption
@ -162,14 +162,14 @@ case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketMe
case class Bind(endpoint: String) extends SocketConnectOption case class Bind(endpoint: String) extends SocketConnectOption
/** /**
* The [[akka.zeromq.Subscribe]] option shall establish a new message filter on a [[akka.zeromq.SocketType.Pub]] socket. * The [[akka.zeromq.Subscribe]] option establishes a new message filter on a [[akka.zeromq.SocketType.Pub]] socket.
* Newly created [[akka.zeromq.SocketType.Sub]] sockets shall filter out all incoming messages, * Newly created [[akka.zeromq.SocketType.Sub]] sockets filter out all incoming messages,
* therefore you should send this option to establish an initial message filter. * therefore you should send this option to establish an initial message filter.
* *
* An empty payload of length zero shall subscribe to all incoming messages. * An empty payload of length zero will subscribe to all incoming messages.
* A non-empty payload shall subscribe to all messages beginning with the specified prefix. * A non-empty payload will subscribe to all messages beginning with the specified prefix.
* Multiple filters may be attached to a single [[akka.zeromq.SocketType.Sub]] socket, * Multiple filters may be attached to a single [[akka.zeromq.SocketType.Sub]] socket,
* in which case a message shall be accepted if it matches at least one filter. * in which case a message will be accepted if it matches at least one filter.
* *
* @param payload the topic to subscribe to * @param payload the topic to subscribe to
*/ */

View file

@ -4,6 +4,7 @@
package akka.zeromq package akka.zeromq
import org.zeromq.{ ZMQ JZMQ } import org.zeromq.{ ZMQ JZMQ }
import org.zeromq.ZMQ.Poller
import akka.actor._ import akka.actor._
import akka.pattern.ask import akka.pattern.ask
import scala.concurrent.Await import scala.concurrent.Await
@ -44,6 +45,8 @@ class ZeroMQExtension(system: ActorSystem) extends Extension {
val DefaultPollTimeout: Duration = Duration(system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS) val DefaultPollTimeout: Duration = Duration(system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS)
val NewSocketTimeout: Timeout = Timeout(Duration(system.settings.config.getMilliseconds("akka.zeromq.new-socket-timeout"), TimeUnit.MILLISECONDS)) val NewSocketTimeout: Timeout = Timeout(Duration(system.settings.config.getMilliseconds("akka.zeromq.new-socket-timeout"), TimeUnit.MILLISECONDS))
val pollTimeUnit = if (version.major >= 3) TimeUnit.MILLISECONDS else TimeUnit.MICROSECONDS
/** /**
* The version of the ZeroMQ library * The version of the ZeroMQ library
* @return a [[akka.zeromq.ZeroMQVersion]] * @return a [[akka.zeromq.ZeroMQVersion]]

View file

@ -328,7 +328,8 @@ object AkkaBuild extends Build {
settings = defaultSettings ++ Sphinx.settings ++ Seq( settings = defaultSettings ++ Sphinx.settings ++ Seq(
unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get },
libraryDependencies ++= Dependencies.docs, libraryDependencies ++= Dependencies.docs,
unmanagedSourceDirectories in ScalariformKeys.format in Test <<= unmanagedSourceDirectories in Test unmanagedSourceDirectories in ScalariformKeys.format in Test <<= unmanagedSourceDirectories in Test,
testOptions += Tests.Argument(TestFrameworks.JUnit, "-v")
) )
) )
@ -515,7 +516,7 @@ object Dependencies {
val tutorials = Seq(Test.scalatest, Test.junit) val tutorials = Seq(Test.scalatest, Test.junit)
val docs = Seq(Test.scalatest, Test.junit, Test.specs2) val docs = Seq(Test.scalatest, Test.junit, Test.specs2, Test.junitIntf)
val zeroMQ = Seq(protobuf, Dependency.zeroMQ, Test.scalatest, Test.junit) val zeroMQ = Seq(protobuf, Dependency.zeroMQ, Test.scalatest, Test.junit)
} }
@ -550,6 +551,8 @@ object Dependency {
val ariesProxy = "org.apache.aries.proxy" % "org.apache.aries.proxy.impl" % "0.3" % "test" // ApacheV2 val ariesProxy = "org.apache.aries.proxy" % "org.apache.aries.proxy.impl" % "0.3" % "test" // ApacheV2
val pojosr = "com.googlecode.pojosr" % "de.kalpatec.pojosr.framework" % "0.1.4" % "test" // ApacheV2 val pojosr = "com.googlecode.pojosr" % "de.kalpatec.pojosr.framework" % "0.1.4" % "test" // ApacheV2
val tinybundles = "org.ops4j.pax.tinybundles" % "tinybundles" % "1.0.0" % "test" // ApacheV2 val tinybundles = "org.ops4j.pax.tinybundles" % "tinybundles" % "1.0.0" % "test" // ApacheV2
val log4j = "log4j" % "log4j" % "1.2.14" % "test" // ApacheV2
val junitIntf = "com.novocode" % "junit-interface" % "0.8" % "test" // MIT
} }
} }