Merging in master, huge work trying to get things to compile, tests not green at this stage
This commit is contained in:
commit
ac5b5de90a
68 changed files with 3759 additions and 2144 deletions
1
.gitignore
vendored
1
.gitignore
vendored
|
|
@ -64,3 +64,4 @@ mongoDB/
|
||||||
redis/
|
redis/
|
||||||
beanstalk/
|
beanstalk/
|
||||||
.scalastyle
|
.scalastyle
|
||||||
|
bin/
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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 (it’s due to shutdown)
|
failing must not be true // because once failing => always failing (it’s 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())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
11
akka-actor/src/main/java/akka/japi/JAPI.java
Normal file
11
akka-actor/src/main/java/akka/japi/JAPI.java
Normal 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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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.
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
905
akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala
Normal file
905
akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala
Normal 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)
|
||||||
|
}
|
||||||
128
akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala
Normal file
128
akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala
Normal 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
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
107
akka-cluster/src/main/scala/akka/cluster/ClusterJmx.scala
Normal file
107
akka-cluster/src/main/scala/akka/cluster/ClusterJmx.scala
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
212
akka-cluster/src/main/scala/akka/cluster/Gossip.scala
Normal file
212
akka-cluster/src/main/scala/akka/cluster/Gossip.scala
Normal 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
|
||||||
|
|
||||||
117
akka-cluster/src/main/scala/akka/cluster/Member.scala
Normal file
117
akka-cluster/src/main/scala/akka/cluster/Member.scala
Normal 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
|
||||||
|
}
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
|
|
@ -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()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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:
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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");
|
||||||
|
|
|
||||||
|
|
@ -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);
|
||||||
|
|
|
||||||
|
|
@ -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);
|
||||||
|
|
|
||||||
399
akka-docs/java/code/docs/testkit/TestKitDocTest.java
Normal file
399
akka-docs/java/code/docs/testkit/TestKitDocTest.java
Normal 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 probe’s 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
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
95
akka-docs/java/code/docs/testkit/TestKitSampleTest.java
Normal file
95
akka-docs/java/code/docs/testkit/TestKitSampleTest.java
Normal 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
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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 actor’s 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 Java’s 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 Scala’s
|
||||||
|
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 Java’s inability to infer the class’s
|
||||||
|
type argument.
|
||||||
|
|
||||||
|
* **AwaitCond**
|
||||||
|
|
||||||
|
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-awaitCond
|
||||||
|
|
||||||
|
This general construct is not connected with the test kit’s 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 forwarder’s 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 system’s 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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
================
|
================
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
================
|
================
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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 }
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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) }
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
-----------------
|
-----------------
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
=========================
|
=========================
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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"
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -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,
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
329
akka-testkit/src/main/java/akka/testkit/JavaTestKit.java
Normal file
329
akka-testkit/src/main/java/akka/testkit/JavaTestKit.java
Normal 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -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]]
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue