Merged with master
This commit is contained in:
commit
cb0cfac6c7
96 changed files with 2377 additions and 1965 deletions
|
|
@ -6,7 +6,7 @@ package akka.actor
|
|||
import akka.testkit._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.testkit.TestEvent._
|
||||
import akka.dispatch.{ Await, MessageQueueAppendFailedException, BoundedDequeBasedMailbox }
|
||||
import akka.dispatch.{ Await, BoundedDequeBasedMailbox }
|
||||
import akka.pattern.ask
|
||||
import akka.util.duration._
|
||||
import akka.actor.ActorSystem.Settings
|
||||
|
|
@ -17,16 +17,8 @@ object ActorWithBoundedStashSpec {
|
|||
|
||||
class StashingActor(implicit sys: ActorSystem) extends Actor with Stash {
|
||||
def receive = {
|
||||
case "hello" ⇒
|
||||
stash()
|
||||
sender ! "OK"
|
||||
case "world" ⇒
|
||||
try {
|
||||
unstashAll()
|
||||
} catch {
|
||||
case e: MessageQueueAppendFailedException ⇒
|
||||
expectedException.open()
|
||||
}
|
||||
case "hello" ⇒ stash()
|
||||
case "world" ⇒ unstashAll()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -36,18 +28,10 @@ object ActorWithBoundedStashSpec {
|
|||
def receive = {
|
||||
case "hello" ⇒
|
||||
numStashed += 1
|
||||
try {
|
||||
stash()
|
||||
} catch {
|
||||
case e: StashOverflowException ⇒
|
||||
if (numStashed == 21) stashOverflow.open()
|
||||
}
|
||||
try stash() catch { case e: StashOverflowException ⇒ if (numStashed == 21) sender ! "STASHOVERFLOW" }
|
||||
}
|
||||
}
|
||||
|
||||
@volatile var expectedException: TestLatch = null
|
||||
@volatile var stashOverflow: TestLatch = null
|
||||
|
||||
val testConf: Config = ConfigFactory.parseString("""
|
||||
my-dispatcher {
|
||||
mailbox-type = "akka.actor.ActorWithBoundedStashSpec$Bounded"
|
||||
|
|
@ -56,47 +40,42 @@ object ActorWithBoundedStashSpec {
|
|||
""")
|
||||
|
||||
// bounded deque-based mailbox with capacity 10
|
||||
class Bounded(settings: Settings, config: Config) extends BoundedDequeBasedMailbox(10, 5 seconds)
|
||||
|
||||
class Bounded(settings: Settings, config: Config) extends BoundedDequeBasedMailbox(10, 1 seconds)
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ActorWithBoundedStashSpec extends AkkaSpec(ActorWithBoundedStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach {
|
||||
class ActorWithBoundedStashSpec extends AkkaSpec(ActorWithBoundedStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach with ImplicitSender {
|
||||
import ActorWithBoundedStashSpec._
|
||||
|
||||
implicit val sys = system
|
||||
|
||||
override def atStartup {
|
||||
system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
|
||||
}
|
||||
override def atStartup { system.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) }
|
||||
|
||||
def myProps(creator: ⇒ Actor): Props = Props(creator).withDispatcher("my-dispatcher")
|
||||
|
||||
"An Actor with Stash and BoundedDequeBasedMailbox" must {
|
||||
|
||||
"throw a MessageQueueAppendFailedException in case of a capacity violation" in {
|
||||
ActorWithBoundedStashSpec.expectedException = new TestLatch
|
||||
"end up in DeadLetters in case of a capacity violation" in {
|
||||
system.eventStream.subscribe(testActor, classOf[DeadLetter])
|
||||
|
||||
val stasher = system.actorOf(myProps(new StashingActor))
|
||||
// fill up stash
|
||||
val futures = for (_ ← 1 to 11) yield { stasher ? "hello" }
|
||||
futures foreach { Await.ready(_, 10 seconds) }
|
||||
(1 to 11) foreach { _ ⇒ stasher ! "hello" }
|
||||
|
||||
// cause unstashAll with capacity violation
|
||||
stasher ! "world"
|
||||
Await.ready(ActorWithBoundedStashSpec.expectedException, 10 seconds)
|
||||
expectMsg(DeadLetter("hello", testActor, stasher))
|
||||
system.eventStream.unsubscribe(testActor, classOf[DeadLetter])
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"An Actor with bounded Stash" must {
|
||||
|
||||
"throw a StashOverflowException in case of a stash capacity violation" in {
|
||||
ActorWithBoundedStashSpec.stashOverflow = new TestLatch
|
||||
val stasher = system.actorOf(myProps(new StashingActorWithOverflow))
|
||||
// fill up stash
|
||||
for (_ ← 1 to 21) { stasher ! "hello" }
|
||||
Await.ready(ActorWithBoundedStashSpec.stashOverflow, 10 seconds)
|
||||
(1 to 21) foreach { _ ⇒ stasher ! "hello" }
|
||||
expectMsg("STASHOVERFLOW")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
|
|||
"notify with one Terminated message when an Actor is stopped" in {
|
||||
val terminal = system.actorOf(Props.empty)
|
||||
startWatching(terminal) ! "hallo"
|
||||
expectMsg("hallo") // this ensures that the DaemonMsgWatch has been received before we send the PoisonPill
|
||||
expectMsg("hallo")
|
||||
|
||||
terminal ! PoisonPill
|
||||
|
||||
|
|
|
|||
|
|
@ -339,9 +339,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
|
|||
OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 10 seconds)(classOf[Exception] :: Nil))))
|
||||
|
||||
val dyingProps = Props(new Actor {
|
||||
inits.incrementAndGet
|
||||
|
||||
if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!")
|
||||
if (inits.incrementAndGet % 2 == 0) throw new IllegalStateException("Don't wanna!")
|
||||
|
||||
def receive = {
|
||||
case Ping ⇒ sender ! PongMessage
|
||||
|
|
@ -366,5 +364,39 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
|
|||
|
||||
system.stop(supervisor)
|
||||
}
|
||||
|
||||
"must not lose system messages when a NonFatal exception occurs when processing a system message" in {
|
||||
val parent = system.actorOf(Props(new Actor {
|
||||
override val supervisorStrategy = OneForOneStrategy()({
|
||||
case e: IllegalStateException if e.getMessage == "OHNOES" ⇒ throw e
|
||||
case _ ⇒ SupervisorStrategy.Restart
|
||||
})
|
||||
val child = context.watch(context.actorOf(Props(new Actor {
|
||||
override def postRestart(reason: Throwable): Unit = testActor ! "child restarted"
|
||||
def receive = {
|
||||
case "die" ⇒ throw new IllegalStateException("OHNOES")
|
||||
case "test" ⇒ sender ! "child green"
|
||||
}
|
||||
}), "child"))
|
||||
|
||||
override def postRestart(reason: Throwable): Unit = testActor ! "parent restarted"
|
||||
|
||||
def receive = {
|
||||
case t @ Terminated(`child`) ⇒ testActor ! "child terminated"
|
||||
case "die" ⇒ child ! "die"
|
||||
case "test" ⇒ sender ! "green"
|
||||
case "testchild" ⇒ child forward "test"
|
||||
}
|
||||
}))
|
||||
|
||||
parent ! "die"
|
||||
parent ! "testchild"
|
||||
expectMsg("parent restarted")
|
||||
expectMsg("child terminated")
|
||||
parent ! "test"
|
||||
expectMsg("green")
|
||||
parent ! "testchild"
|
||||
expectMsg("child green")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -374,7 +374,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
|||
def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path compareTo rr.self.path }
|
||||
} foreach {
|
||||
case cell: ActorCell ⇒
|
||||
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain()))
|
||||
System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain(null)))
|
||||
}
|
||||
|
||||
System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages)
|
||||
|
|
|
|||
|
|
@ -6,9 +6,8 @@ import java.util.concurrent.ConcurrentLinkedQueue
|
|||
import akka.util._
|
||||
import akka.util.duration._
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef }
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor._
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
|
||||
|
|
@ -39,9 +38,10 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
|
|||
q.numberOfMessages must be === config.capacity
|
||||
q.hasMessages must be === true
|
||||
|
||||
intercept[MessageQueueAppendFailedException] {
|
||||
q.enqueue(null, exampleMessage)
|
||||
}
|
||||
system.eventStream.subscribe(testActor, classOf[DeadLetter])
|
||||
q.enqueue(testActor, exampleMessage)
|
||||
expectMsg(DeadLetter(exampleMessage.message, system.deadLetters, testActor))
|
||||
system.eventStream.unsubscribe(testActor, classOf[DeadLetter])
|
||||
|
||||
q.dequeue must be === exampleMessage
|
||||
q.numberOfMessages must be(config.capacity - 1)
|
||||
|
|
|
|||
|
|
@ -73,7 +73,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
watch(router)
|
||||
watch(c2)
|
||||
system.stop(c2)
|
||||
expectMsg(Terminated(c2))
|
||||
expectMsgPF() {
|
||||
case t @ Terminated(`c2`) if t.existenceConfirmed == true ⇒ t
|
||||
}
|
||||
// it might take a while until the Router has actually processed the Terminated message
|
||||
awaitCond {
|
||||
router ! ""
|
||||
|
|
@ -84,7 +86,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
res == Seq(c1, c1)
|
||||
}
|
||||
system.stop(c1)
|
||||
expectMsg(Terminated(router))
|
||||
expectMsgPF() {
|
||||
case t @ Terminated(`router`) if t.existenceConfirmed == true ⇒ t
|
||||
}
|
||||
}
|
||||
|
||||
"be able to send their routees" in {
|
||||
|
|
|
|||
19
akka-actor/src/main/java/akka/actor/AbstractActorCell.java
Normal file
19
akka-actor/src/main/java/akka/actor/AbstractActorCell.java
Normal file
|
|
@ -0,0 +1,19 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.actor;
|
||||
|
||||
import akka.util.Unsafe;
|
||||
|
||||
final class AbstractActorCell {
|
||||
final static long mailboxOffset;
|
||||
|
||||
static {
|
||||
try {
|
||||
mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_mailboxDoNotCallMeDirectly"));
|
||||
} catch(Throwable t){
|
||||
throw new ExceptionInInitializerError(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -8,10 +8,12 @@ import akka.util.Unsafe;
|
|||
|
||||
final class AbstractPromiseActorRef {
|
||||
final static long stateOffset;
|
||||
final static long watchedByOffset;
|
||||
|
||||
static {
|
||||
try {
|
||||
stateOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_stateDoNotCallMeDirectly"));
|
||||
watchedByOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_watchedByDoNotCallMeDirectly"));
|
||||
} catch(Throwable t){
|
||||
throw new ExceptionInInitializerError(t);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -59,7 +59,7 @@ case object Kill extends Kill {
|
|||
/**
|
||||
* When Death Watch is used, the watcher will receive a Terminated(watched) message when watched is terminated.
|
||||
*/
|
||||
case class Terminated(@BeanProperty actor: ActorRef) extends PossiblyHarmful
|
||||
case class Terminated(@BeanProperty actor: ActorRef)(@BeanProperty val existenceConfirmed: Boolean)
|
||||
|
||||
abstract class ReceiveTimeout extends PossiblyHarmful
|
||||
|
||||
|
|
|
|||
|
|
@ -6,16 +6,15 @@ package akka.actor
|
|||
|
||||
import akka.dispatch._
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.{ Stack, TreeMap }
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import akka.event.Logging.{ Debug, Warning, Error }
|
||||
import akka.util.{ Duration, Helpers }
|
||||
import akka.japi.Procedure
|
||||
import java.io.{ NotSerializableException, ObjectOutputStream }
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.util.NonFatal
|
||||
import akka.event.Logging.LogEventException
|
||||
import collection.immutable.{ TreeSet, Stack, TreeMap }
|
||||
import akka.util.{ Unsafe, Duration, Helpers, NonFatal }
|
||||
|
||||
//TODO: everything here for current compatibility - could be limited more
|
||||
|
||||
|
|
@ -187,6 +186,8 @@ private[akka] object ActorCell {
|
|||
|
||||
final val behaviorStackPlaceHolder: Stack[Actor.Receive] = Stack.empty.push(Actor.emptyBehavior)
|
||||
|
||||
final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty
|
||||
|
||||
sealed trait SuspendReason
|
||||
case object UserRequest extends SuspendReason
|
||||
case class Recreation(cause: Throwable) extends SuspendReason
|
||||
|
|
@ -315,9 +316,8 @@ private[akka] class ActorCell(
|
|||
val system: ActorSystemImpl,
|
||||
val self: InternalActorRef,
|
||||
val props: Props,
|
||||
@volatile var parent: InternalActorRef,
|
||||
/*no member*/ _receiveTimeout: Option[Duration]) extends UntypedActorContext {
|
||||
|
||||
@volatile var parent: InternalActorRef) extends UntypedActorContext {
|
||||
import AbstractActorCell.mailboxOffset
|
||||
import ActorCell._
|
||||
|
||||
final def systemImpl = system
|
||||
|
|
@ -350,8 +350,7 @@ private[akka] class ActorCell(
|
|||
/**
|
||||
* In milliseconds
|
||||
*/
|
||||
var receiveTimeoutData: (Long, Cancellable) =
|
||||
if (_receiveTimeout.isDefined) (_receiveTimeout.get.toMillis, emptyCancellable) else emptyReceiveTimeoutData
|
||||
var receiveTimeoutData: (Long, Cancellable) = emptyReceiveTimeoutData
|
||||
|
||||
@volatile
|
||||
var childrenRefs: ChildrenContainer = EmptyChildrenContainer
|
||||
|
|
@ -407,16 +406,13 @@ private[akka] class ActorCell(
|
|||
actor.asInstanceOf[InternalActorRef].stop()
|
||||
}
|
||||
|
||||
var currentMessage: Envelope = null
|
||||
|
||||
var currentMessage: Envelope = _
|
||||
var actor: Actor = _
|
||||
|
||||
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
|
||||
|
||||
@volatile //This must be volatile since it isn't protected by the mailbox status
|
||||
var mailbox: Mailbox = _
|
||||
|
||||
@volatile var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
|
||||
var nextNameSequence: Long = 0
|
||||
var watching: Set[ActorRef] = emptyActorRefSet
|
||||
var watchedBy: Set[ActorRef] = emptyActorRefSet
|
||||
|
||||
//Not thread safe, so should only be used inside the actor that inhabits this ActorCell
|
||||
final protected def randomName(): String = {
|
||||
|
|
@ -428,6 +424,24 @@ private[akka] class ActorCell(
|
|||
@inline
|
||||
final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Returns a reference to the current mailbox
|
||||
*/
|
||||
@inline final def mailbox: Mailbox = Unsafe.instance.getObjectVolatile(this, mailboxOffset).asInstanceOf[Mailbox]
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* replaces the current mailbox using getAndSet semantics
|
||||
*/
|
||||
@tailrec final def swapMailbox(newMailbox: Mailbox): Mailbox = {
|
||||
val oldMailbox = mailbox
|
||||
if (!Unsafe.instance.compareAndSwapObject(this, mailboxOffset, oldMailbox, newMailbox)) swapMailbox(newMailbox)
|
||||
else oldMailbox
|
||||
}
|
||||
|
||||
/**
|
||||
* UntypedActorContext impl
|
||||
*/
|
||||
|
|
@ -440,7 +454,7 @@ private[akka] class ActorCell(
|
|||
* Create the mailbox and enqueue the Create() message to ensure that
|
||||
* this is processed before anything else.
|
||||
*/
|
||||
mailbox = dispatcher.createMailbox(this)
|
||||
swapMailbox(dispatcher.createMailbox(this))
|
||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
mailbox.systemEnqueue(self, Create())
|
||||
|
||||
|
|
@ -460,16 +474,22 @@ private[akka] class ActorCell(
|
|||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
final def stop(): Unit = dispatcher.systemDispatch(this, Terminate())
|
||||
|
||||
override final def watch(subject: ActorRef): ActorRef = {
|
||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
dispatcher.systemDispatch(this, Link(subject))
|
||||
subject
|
||||
override final def watch(subject: ActorRef): ActorRef = subject match {
|
||||
case a: InternalActorRef ⇒
|
||||
if (a != self && !watching.contains(a)) {
|
||||
a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
watching += a
|
||||
}
|
||||
a
|
||||
}
|
||||
|
||||
override final def unwatch(subject: ActorRef): ActorRef = {
|
||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
dispatcher.systemDispatch(this, Unlink(subject))
|
||||
subject
|
||||
override final def unwatch(subject: ActorRef): ActorRef = subject match {
|
||||
case a: InternalActorRef ⇒
|
||||
if (a != self && watching.contains(a)) {
|
||||
a.sendSystemMessage(Unwatch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
watching -= a
|
||||
}
|
||||
a
|
||||
}
|
||||
|
||||
final def children: Iterable[ActorRef] = childrenRefs.children
|
||||
|
|
@ -477,10 +497,8 @@ private[akka] class ActorCell(
|
|||
/**
|
||||
* Impl UntypedActorContext
|
||||
*/
|
||||
final def getChildren(): java.lang.Iterable[ActorRef] = {
|
||||
import scala.collection.JavaConverters.asJavaIterableConverter
|
||||
asJavaIterableConverter(children).asJava
|
||||
}
|
||||
final def getChildren(): java.lang.Iterable[ActorRef] =
|
||||
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
|
||||
|
||||
final def tell(message: Any, sender: ActorRef): Unit =
|
||||
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)(system))
|
||||
|
|
@ -567,15 +585,35 @@ private[akka] class ActorCell(
|
|||
|
||||
def resume(): Unit = if (isNormal) dispatcher resume this
|
||||
|
||||
def link(subject: ActorRef): Unit = if (!isTerminating) {
|
||||
if (system.deathWatch.subscribe(self, subject)) {
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + subject))
|
||||
def addWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
|
||||
val watcheeSelf = watchee == self
|
||||
val watcherSelf = watcher == self
|
||||
|
||||
if (watcheeSelf && !watcherSelf) {
|
||||
if (!watchedBy.contains(watcher)) {
|
||||
watchedBy += watcher
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + watcher))
|
||||
}
|
||||
} else if (!watcheeSelf && watcherSelf) {
|
||||
watch(watchee)
|
||||
} else {
|
||||
system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, self)))
|
||||
}
|
||||
}
|
||||
|
||||
def unlink(subject: ActorRef): Unit = if (!isTerminating) {
|
||||
if (system.deathWatch.unsubscribe(self, subject)) {
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + subject))
|
||||
def remWatcher(watchee: ActorRef, watcher: ActorRef): Unit = {
|
||||
val watcheeSelf = watchee == self
|
||||
val watcherSelf = watcher == self
|
||||
|
||||
if (watcheeSelf && !watcherSelf) {
|
||||
if (watchedBy.contains(watcher)) {
|
||||
watchedBy -= watcher
|
||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + watcher))
|
||||
}
|
||||
} else if (!watcheeSelf && watcherSelf) {
|
||||
unwatch(watchee)
|
||||
} else {
|
||||
system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, self)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -603,15 +641,15 @@ private[akka] class ActorCell(
|
|||
|
||||
try {
|
||||
message match {
|
||||
case Create() ⇒ create()
|
||||
case Recreate(cause) ⇒ recreate(cause)
|
||||
case Link(subject) ⇒ link(subject)
|
||||
case Unlink(subject) ⇒ unlink(subject)
|
||||
case Suspend() ⇒ suspend()
|
||||
case Resume() ⇒ resume()
|
||||
case Terminate() ⇒ terminate()
|
||||
case Supervise(child) ⇒ supervise(child)
|
||||
case ChildTerminated(child) ⇒ handleChildTerminated(child)
|
||||
case Create() ⇒ create()
|
||||
case Recreate(cause) ⇒ recreate(cause)
|
||||
case Watch(watchee, watcher) ⇒ addWatcher(watchee, watcher)
|
||||
case Unwatch(watchee, watcher) ⇒ remWatcher(watchee, watcher)
|
||||
case Suspend() ⇒ suspend()
|
||||
case Resume() ⇒ resume()
|
||||
case Terminate() ⇒ terminate()
|
||||
case Supervise(child) ⇒ supervise(child)
|
||||
case ChildTerminated(child) ⇒ handleChildTerminated(child)
|
||||
}
|
||||
} catch {
|
||||
case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(e, "error while processing " + message)
|
||||
|
|
@ -633,7 +671,7 @@ private[akka] class ActorCell(
|
|||
checkReceiveTimeout // Reschedule receive timeout
|
||||
}
|
||||
|
||||
private final def handleInvokeFailure(t: Throwable, message: String): Unit = try {
|
||||
final def handleInvokeFailure(t: Throwable, message: String): Unit = try {
|
||||
dispatcher.reportFailure(new LogEventException(Error(t, self.path.toString, clazz(actor), message), t))
|
||||
// prevent any further messages to be processed until the actor has been restarted
|
||||
dispatcher.suspend(this)
|
||||
|
|
@ -698,11 +736,32 @@ private[akka] class ActorCell(
|
|||
} finally {
|
||||
try {
|
||||
parent.sendSystemMessage(ChildTerminated(self))
|
||||
system.deathWatch.publish(Terminated(self))
|
||||
|
||||
if (!watchedBy.isEmpty) {
|
||||
val terminated = Terminated(self)(existenceConfirmed = true)
|
||||
try {
|
||||
watchedBy foreach {
|
||||
watcher ⇒
|
||||
try watcher.tell(terminated, self) catch {
|
||||
case NonFatal(t) ⇒ system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch"))
|
||||
}
|
||||
}
|
||||
} finally watchedBy = emptyActorRefSet
|
||||
}
|
||||
|
||||
if (!watching.isEmpty) {
|
||||
try {
|
||||
watching foreach { // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
case watchee: InternalActorRef ⇒ try watchee.sendSystemMessage(Unwatch(watchee, self)) catch {
|
||||
case NonFatal(t) ⇒ system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch"))
|
||||
}
|
||||
}
|
||||
} finally watching = emptyActorRefSet
|
||||
}
|
||||
if (system.settings.DebugLifecycle)
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped"))
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(a), "stopped"))
|
||||
} finally {
|
||||
behaviorStack = ActorCell.behaviorStackPlaceHolder
|
||||
behaviorStack = behaviorStackPlaceHolder
|
||||
clearActorFields(a)
|
||||
actor = null
|
||||
}
|
||||
|
|
|
|||
|
|
@ -227,9 +227,7 @@ private[akka] class LocalActorRef private[akka] (
|
|||
_system: ActorSystemImpl,
|
||||
_props: Props,
|
||||
_supervisor: InternalActorRef,
|
||||
override val path: ActorPath,
|
||||
val systemService: Boolean = false,
|
||||
_receiveTimeout: Option[Duration] = None)
|
||||
override val path: ActorPath)
|
||||
extends InternalActorRef with LocalRef {
|
||||
|
||||
/*
|
||||
|
|
@ -242,16 +240,11 @@ private[akka] class LocalActorRef private[akka] (
|
|||
* us to use purely factory methods for creating LocalActorRefs.
|
||||
*/
|
||||
@volatile
|
||||
private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout)
|
||||
private var actorCell = newActorCell(_system, this, _props, _supervisor)
|
||||
actorCell.start()
|
||||
|
||||
protected def newActorCell(
|
||||
system: ActorSystemImpl,
|
||||
ref: InternalActorRef,
|
||||
props: Props,
|
||||
supervisor: InternalActorRef,
|
||||
receiveTimeout: Option[Duration]): ActorCell =
|
||||
new ActorCell(system, ref, props, supervisor, receiveTimeout)
|
||||
protected def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell =
|
||||
new ActorCell(system, ref, props, supervisor)
|
||||
|
||||
protected def actorContext: ActorContext = actorCell
|
||||
|
||||
|
|
@ -409,16 +402,26 @@ private[akka] object DeadLetterActorRef {
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class EmptyLocalActorRef(
|
||||
override val provider: ActorRefProvider,
|
||||
override val path: ActorPath,
|
||||
val eventStream: EventStream) extends MinimalActorRef {
|
||||
private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
|
||||
override val path: ActorPath,
|
||||
val eventStream: EventStream) extends MinimalActorRef {
|
||||
|
||||
override def isTerminated(): Boolean = true
|
||||
|
||||
override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message)
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
|
||||
case d: DeadLetter ⇒ // do NOT form endless loops, since deadLetters will resend!
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
case d: DeadLetter ⇒ specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend!
|
||||
case _ ⇒ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
|
||||
protected def specialHandle(msg: Any): Boolean = msg match {
|
||||
case w: Watch ⇒
|
||||
if (w.watchee == this && w.watcher != this)
|
||||
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false)
|
||||
true
|
||||
case _: Unwatch ⇒ true // Just ignore
|
||||
case _ ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -428,12 +431,22 @@ private[akka] class EmptyLocalActorRef(
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream)
|
||||
extends EmptyLocalActorRef(_provider, _path, _eventStream) {
|
||||
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider,
|
||||
_path: ActorPath,
|
||||
_eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) {
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
|
||||
case d: DeadLetter ⇒ eventStream.publish(d)
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
case d: DeadLetter ⇒ if (!specialHandle(d.message)) eventStream.publish(d)
|
||||
case _ ⇒ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
|
||||
override protected def specialHandle(msg: Any): Boolean = msg match {
|
||||
case w: Watch ⇒
|
||||
if (w.watchee != this && w.watcher != this)
|
||||
w.watcher ! Terminated(w.watchee)(existenceConfirmed = false)
|
||||
true
|
||||
case w: Unwatch ⇒ true // Just ignore
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
|
|
|
|||
|
|
@ -8,8 +8,8 @@ import java.util.concurrent.atomic.AtomicLong
|
|||
import akka.dispatch._
|
||||
import akka.routing._
|
||||
import akka.AkkaException
|
||||
import akka.util.{ Switch, Helpers }
|
||||
import akka.event._
|
||||
import akka.util.{ NonFatal, Switch, Helpers }
|
||||
|
||||
/**
|
||||
* Interface for all ActorRef providers to implement.
|
||||
|
|
@ -38,11 +38,6 @@ trait ActorRefProvider {
|
|||
*/
|
||||
def deadLetters: ActorRef
|
||||
|
||||
/**
|
||||
* Reference to the death watch service.
|
||||
*/
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
/**
|
||||
* The root path for all actors within this actor system, including remote
|
||||
* address if enabled.
|
||||
|
|
@ -162,8 +157,9 @@ trait ActorRefFactory {
|
|||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected def provider: ActorRefProvider
|
||||
|
||||
/**
|
||||
* Returns the default MessageDispatcher used by this ActorRefFactory
|
||||
* Returns the default MessageDispatcher associated with this ActorRefFactory
|
||||
*/
|
||||
implicit def dispatcher: MessageDispatcher
|
||||
|
||||
|
|
@ -339,8 +335,6 @@ class LocalActorRefProvider(
|
|||
|
||||
override val deadLetters: InternalActorRef = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
|
||||
|
||||
override val deathWatch: DeathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
|
|
@ -379,9 +373,9 @@ class LocalActorRefProvider(
|
|||
|
||||
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
|
||||
message match {
|
||||
case Supervise(child) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead
|
||||
case ChildTerminated(child) ⇒ stop()
|
||||
case _ ⇒ log.error(this + " received unexpected system message [" + message + "]")
|
||||
case Supervise(_) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead
|
||||
case ChildTerminated(_) ⇒ stop()
|
||||
case _ ⇒ log.error(this + " received unexpected system message [" + message + "]")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -409,8 +403,8 @@ class LocalActorRefProvider(
|
|||
|
||||
def receive = {
|
||||
case Terminated(_) ⇒ context.stop(self)
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case NonFatal(e) ⇒ Status.Failure(e) })
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case NonFatal(e) ⇒ Status.Failure(e) })
|
||||
case StopChild(child) ⇒ context.stop(child); sender ! "ok"
|
||||
case m ⇒ deadLetters ! DeadLetter(m, sender, self)
|
||||
}
|
||||
|
|
@ -441,8 +435,8 @@ class LocalActorRefProvider(
|
|||
|
||||
def receive = {
|
||||
case Terminated(_) ⇒ eventStream.stopDefaultLoggers(); context.stop(self)
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case NonFatal(e) ⇒ Status.Failure(e) })
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case NonFatal(e) ⇒ Status.Failure(e) })
|
||||
case StopChild(child) ⇒ context.stop(child); sender ! "ok"
|
||||
case m ⇒ deadLetters ! DeadLetter(m, sender, self)
|
||||
}
|
||||
|
|
@ -479,19 +473,11 @@ class LocalActorRefProvider(
|
|||
private val guardianProps = Props(new Guardian)
|
||||
|
||||
lazy val rootGuardian: InternalActorRef =
|
||||
new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) {
|
||||
object Extra {
|
||||
def unapply(s: String): Option[InternalActorRef] = extraNames.get(s)
|
||||
}
|
||||
|
||||
new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath) {
|
||||
override def getParent: InternalActorRef = this
|
||||
|
||||
override def getSingleChild(name: String): InternalActorRef = {
|
||||
name match {
|
||||
case "temp" ⇒ tempContainer
|
||||
case Extra(e) ⇒ e
|
||||
case _ ⇒ super.getSingleChild(name)
|
||||
}
|
||||
override def getSingleChild(name: String): InternalActorRef = name match {
|
||||
case "temp" ⇒ tempContainer
|
||||
case other ⇒ extraNames.get(other).getOrElse(super.getSingleChild(other))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -516,8 +502,8 @@ class LocalActorRefProvider(
|
|||
def init(_system: ActorSystemImpl) {
|
||||
system = _system
|
||||
// chain death watchers so that killing guardian stops the application
|
||||
deathWatch.subscribe(systemGuardian, guardian)
|
||||
deathWatch.subscribe(rootGuardian, systemGuardian)
|
||||
systemGuardian.sendSystemMessage(Watch(guardian, systemGuardian))
|
||||
rootGuardian.sendSystemMessage(Watch(systemGuardian, rootGuardian))
|
||||
eventStream.startDefaultLoggers(_system)
|
||||
}
|
||||
|
||||
|
|
@ -555,7 +541,7 @@ class LocalActorRefProvider(
|
|||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
||||
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean): InternalActorRef = {
|
||||
props.routerConfig match {
|
||||
case NoRouter ⇒ new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
|
||||
case NoRouter ⇒ new LocalActorRef(system, props, supervisor, path) // create a local actor
|
||||
case router ⇒
|
||||
val lookup = if (lookupDeploy) deployer.lookup(path) else None
|
||||
val fromProps = Iterator(props.deploy.copy(routerConfig = props.deploy.routerConfig withFallback router))
|
||||
|
|
@ -567,18 +553,8 @@ class LocalActorRefProvider(
|
|||
def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None
|
||||
}
|
||||
|
||||
class LocalDeathWatch(val mapSize: Int) extends DeathWatch with ActorClassification {
|
||||
private[akka] class GuardianCell(_system: ActorSystemImpl, _self: InternalActorRef, _props: Props, _parent: InternalActorRef)
|
||||
extends ActorCell(_system, _self, _props, _parent) {
|
||||
|
||||
override def publish(event: Event): Unit = {
|
||||
val monitors = dissociate(classify(event))
|
||||
if (monitors.nonEmpty) monitors.foreach(_ ! event)
|
||||
}
|
||||
|
||||
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = {
|
||||
if (!super.subscribe(subscriber, to)) {
|
||||
subscriber ! Terminated(to)
|
||||
false
|
||||
} else true
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -305,8 +305,9 @@ abstract class ActorSystem extends ActorRefFactory {
|
|||
implicit def dispatcher: MessageDispatcher
|
||||
|
||||
/**
|
||||
* Register a block of code (callback) to run after all actors in this actor system have
|
||||
* been stopped. Multiple code blocks may be registered by calling this method multiple times.
|
||||
* Register a block of code (callback) to run after ActorSystem.shutdown has been issued and
|
||||
* all actors in this actor system have been stopped.
|
||||
* Multiple code blocks may be registered by calling this method multiple times.
|
||||
* The callbacks will be run sequentially in reverse order of registration, i.e.
|
||||
* last registration is run first.
|
||||
*
|
||||
|
|
@ -317,8 +318,9 @@ abstract class ActorSystem extends ActorRefFactory {
|
|||
def registerOnTermination[T](code: ⇒ T): Unit
|
||||
|
||||
/**
|
||||
* Register a block of code (callback) to run after all actors in this actor system have
|
||||
* been stopped. Multiple code blocks may be registered by calling this method multiple times.
|
||||
* Register a block of code (callback) to run after ActorSystem.shutdown has been issued and
|
||||
* all actors in this actor system have been stopped.
|
||||
* Multiple code blocks may be registered by calling this method multiple times.
|
||||
* The callbacks will be run sequentially in reverse order of registration, i.e.
|
||||
* last registration is run first.
|
||||
*
|
||||
|
|
@ -408,11 +410,6 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
|||
*/
|
||||
def systemGuardian: InternalActorRef
|
||||
|
||||
/**
|
||||
* Implementation of the mechanism which is used for watch()/unwatch().
|
||||
*/
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
/**
|
||||
* A ThreadFactory that can be used if the transport needs to create any Threads
|
||||
*/
|
||||
|
|
@ -485,26 +482,17 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
|
||||
private[akka] def systemActorOf(props: Props, name: String): ActorRef = {
|
||||
implicit val timeout = settings.CreationTimeout
|
||||
Await.result(systemGuardian ? CreateChild(props, name), timeout.duration) match {
|
||||
case ref: ActorRef ⇒ ref
|
||||
case ex: Exception ⇒ throw ex
|
||||
}
|
||||
Await.result((systemGuardian ? CreateChild(props, name)).mapTo[ActorRef], timeout.duration)
|
||||
}
|
||||
|
||||
def actorOf(props: Props, name: String): ActorRef = {
|
||||
implicit val timeout = settings.CreationTimeout
|
||||
Await.result(guardian ? CreateChild(props, name), timeout.duration) match {
|
||||
case ref: ActorRef ⇒ ref
|
||||
case ex: Exception ⇒ throw ex
|
||||
}
|
||||
Await.result((guardian ? CreateChild(props, name)).mapTo[ActorRef], timeout.duration)
|
||||
}
|
||||
|
||||
def actorOf(props: Props): ActorRef = {
|
||||
implicit val timeout = settings.CreationTimeout
|
||||
Await.result(guardian ? CreateRandomNameChild(props), timeout.duration) match {
|
||||
case ref: ActorRef ⇒ ref
|
||||
case ex: Exception ⇒ throw ex
|
||||
}
|
||||
Await.result((guardian ? CreateRandomNameChild(props)).mapTo[ActorRef], timeout.duration)
|
||||
}
|
||||
|
||||
def stop(actor: ActorRef): Unit = {
|
||||
|
|
@ -547,7 +535,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
|
||||
//FIXME Why do we need this at all?
|
||||
val deadLetterQueue: MessageQueue = new MessageQueue {
|
||||
def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
|
||||
def enqueue(receiver: ActorRef, envelope: Envelope): Unit =
|
||||
deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver)
|
||||
def dequeue() = null
|
||||
def hasMessages = false
|
||||
def numberOfMessages = 0
|
||||
|
|
@ -556,8 +545,9 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
//FIXME Why do we need this at all?
|
||||
val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) {
|
||||
becomeClosed()
|
||||
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = deadLetters ! DeadLetter(handle, receiver, receiver)
|
||||
def systemDrain(): SystemMessage = null
|
||||
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit =
|
||||
deadLetters ! DeadLetter(handle, receiver, receiver)
|
||||
def systemDrain(newContents: SystemMessage): SystemMessage = null
|
||||
def hasSystemMessages = false
|
||||
}
|
||||
|
||||
|
|
@ -570,7 +560,6 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
def lookupRoot: InternalActorRef = provider.rootGuardian
|
||||
def guardian: InternalActorRef = provider.guardian
|
||||
def systemGuardian: InternalActorRef = provider.systemGuardian
|
||||
def deathWatch: DeathWatch = provider.deathWatch
|
||||
|
||||
def /(actorName: String): ActorPath = guardian.path / actorName
|
||||
def /(path: Iterable[String]): ActorPath = guardian.path / path
|
||||
|
|
|
|||
|
|
@ -102,11 +102,15 @@ private[akka] case class ChildTerminated(child: ActorRef) extends SystemMessage
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch
|
||||
private[akka] case class Watch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to establish a DeathWatch
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch
|
||||
private[akka] case class Unwatch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to tear down a DeathWatch
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case object NoMessage extends SystemMessage // switched into the mailbox to signal termination
|
||||
|
||||
final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () ⇒ Unit) extends Runnable {
|
||||
def run(): Unit =
|
||||
|
|
@ -310,16 +314,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
case 0 ⇒
|
||||
shutdownSchedule match {
|
||||
case UNSCHEDULED ⇒
|
||||
if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) {
|
||||
scheduleShutdownAction()
|
||||
()
|
||||
} else ifSensibleToDoSoThenScheduleShutdown()
|
||||
if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) scheduleShutdownAction()
|
||||
else ifSensibleToDoSoThenScheduleShutdown()
|
||||
case SCHEDULED ⇒
|
||||
if (updateShutdownSchedule(SCHEDULED, RESCHEDULED)) ()
|
||||
else ifSensibleToDoSoThenScheduleShutdown()
|
||||
case RESCHEDULED ⇒ ()
|
||||
case RESCHEDULED ⇒
|
||||
}
|
||||
case _ ⇒ ()
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
private def scheduleShutdownAction(): Unit = {
|
||||
|
|
@ -349,9 +351,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
protected[akka] def unregister(actor: ActorCell) {
|
||||
if (debug) actors.remove(this, actor.self)
|
||||
addInhabitants(-1)
|
||||
val mailBox = actor.mailbox
|
||||
val mailBox = actor.swapMailbox(deadLetterMailbox)
|
||||
mailBox.becomeClosed() // FIXME reschedule in tell if possible race with cleanUp is detected in order to properly clean up
|
||||
actor.mailbox = deadLetterMailbox
|
||||
mailBox.cleanUp()
|
||||
}
|
||||
|
||||
|
|
@ -359,7 +360,6 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
@tailrec
|
||||
final def run() {
|
||||
shutdownSchedule match {
|
||||
case UNSCHEDULED ⇒ ()
|
||||
case SCHEDULED ⇒
|
||||
try {
|
||||
if (inhabitants == 0) shutdown() //Warning, racy
|
||||
|
|
@ -369,6 +369,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
case RESCHEDULED ⇒
|
||||
if (updateShutdownSchedule(RESCHEDULED, SCHEDULED)) scheduleShutdownAction()
|
||||
else run()
|
||||
case UNSCHEDULED ⇒
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -50,17 +50,15 @@ class BalancingDispatcher(
|
|||
|
||||
private class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue {
|
||||
override def cleanUp(): Unit = {
|
||||
val dlq = actor.systemImpl.deadLetterMailbox
|
||||
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
|
||||
if (hasSystemMessages) {
|
||||
val dlq = actor.systemImpl.deadLetterMailbox
|
||||
var message = systemDrain()
|
||||
while (message ne null) {
|
||||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
dlq.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
var message = systemDrain(NoMessage)
|
||||
while (message ne null) {
|
||||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
dlq.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,18 +6,11 @@ package akka.dispatch
|
|||
import akka.AkkaException
|
||||
import java.util.{ Comparator, PriorityQueue, Queue, Deque }
|
||||
import akka.util._
|
||||
import akka.actor.{ ActorCell, ActorRef }
|
||||
import java.util.concurrent._
|
||||
import annotation.tailrec
|
||||
import akka.event.Logging.Error
|
||||
import akka.actor.ActorContext
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
/**
|
||||
* This exception normally is thrown when a bounded mailbox is over capacity
|
||||
*/
|
||||
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
import akka.actor._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -169,6 +162,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
*/
|
||||
protected final def systemQueueGet: SystemMessage =
|
||||
Unsafe.instance.getObjectVolatile(this, AbstractMailbox.systemMessageOffset).asInstanceOf[SystemMessage]
|
||||
|
||||
protected final def systemQueuePut(_old: SystemMessage, _new: SystemMessage): Boolean =
|
||||
Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old, _new)
|
||||
|
||||
|
|
@ -207,21 +201,47 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will at least try to process all queued system messages: in case of
|
||||
* failure simply drop and go on to the next, because there is nothing to
|
||||
* restart here (failure is in ActorCell somewhere …). In case the mailbox
|
||||
* becomes closed (because of processing a Terminate message), dump all
|
||||
* already dequeued message to deadLetters.
|
||||
*/
|
||||
final def processAllSystemMessages() {
|
||||
var nextMessage = systemDrain()
|
||||
try {
|
||||
while ((nextMessage ne null) && !isClosed) {
|
||||
if (debug) println(actor.self + " processing system message " + nextMessage + " with " + actor.childrenRefs)
|
||||
actor systemInvoke nextMessage
|
||||
nextMessage = nextMessage.next
|
||||
// don’t ever execute normal message when system message present!
|
||||
if (nextMessage eq null) nextMessage = systemDrain()
|
||||
var failure: Throwable = null
|
||||
var nextMessage = systemDrain(null)
|
||||
while ((nextMessage ne null) && !isClosed) {
|
||||
val msg = nextMessage
|
||||
nextMessage = nextMessage.next
|
||||
msg.next = null
|
||||
if (debug) println(actor.self + " processing system message " + msg + " with " + actor.childrenRefs)
|
||||
try {
|
||||
actor systemInvoke msg
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
if (failure eq null) failure = e
|
||||
actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system message " + msg + ": " + e.getMessage))
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
|
||||
throw e
|
||||
// don’t ever execute normal message when system message present!
|
||||
if ((nextMessage eq null) && !isClosed) nextMessage = systemDrain(null)
|
||||
}
|
||||
/*
|
||||
* if we closed the mailbox, we must dump the remaining system messages
|
||||
* to deadLetters (this is essential for DeathWatch)
|
||||
*/
|
||||
while (nextMessage ne null) {
|
||||
val msg = nextMessage
|
||||
nextMessage = nextMessage.next
|
||||
msg.next = null
|
||||
try actor.systemImpl.deadLetterMailbox.systemEnqueue(actor.self, msg)
|
||||
catch {
|
||||
case NonFatal(e) ⇒ actor.system.eventStream.publish(
|
||||
Error(e, actor.self.path.toString, this.getClass, "error while enqueuing " + msg + " to deadLetters: " + e.getMessage))
|
||||
}
|
||||
}
|
||||
// if something happened while processing, fail this actor (most probable: exception in supervisorStrategy)
|
||||
if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage)
|
||||
}
|
||||
|
||||
@inline
|
||||
|
|
@ -235,15 +255,13 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
protected[dispatch] def cleanUp(): Unit =
|
||||
if (actor ne null) { // actor is null for the deadLetterMailbox
|
||||
val dlm = actor.systemImpl.deadLetterMailbox
|
||||
if (hasSystemMessages) {
|
||||
var message = systemDrain()
|
||||
while (message ne null) {
|
||||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
dlm.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
var message = systemDrain(NoMessage)
|
||||
while (message ne null) {
|
||||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
dlm.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
|
||||
if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run()
|
||||
|
|
@ -300,7 +318,7 @@ private[akka] trait SystemMessageQueue {
|
|||
/**
|
||||
* Dequeue all messages from system queue and return them as single-linked list.
|
||||
*/
|
||||
def systemDrain(): SystemMessage
|
||||
def systemDrain(newContents: SystemMessage): SystemMessage
|
||||
|
||||
def hasSystemMessages: Boolean
|
||||
}
|
||||
|
|
@ -315,26 +333,30 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
|||
assert(message.next eq null)
|
||||
if (Mailbox.debug) println(actor.self + " having enqueued " + message)
|
||||
val head = systemQueueGet
|
||||
/*
|
||||
* this write is safely published by the compareAndSet contained within
|
||||
* systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec
|
||||
* guarantees that “head” uses the value obtained from systemQueueGet above.
|
||||
* Hence, SystemMessage.next does not need to be volatile.
|
||||
*/
|
||||
message.next = head
|
||||
if (!systemQueuePut(head, message)) {
|
||||
message.next = null
|
||||
systemEnqueue(receiver, message)
|
||||
if (head == NoMessage) actor.system.deadLetterMailbox.systemEnqueue(receiver, message)
|
||||
else {
|
||||
/*
|
||||
* this write is safely published by the compareAndSet contained within
|
||||
* systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec
|
||||
* guarantees that “head” uses the value obtained from systemQueueGet above.
|
||||
* Hence, SystemMessage.next does not need to be volatile.
|
||||
*/
|
||||
message.next = head
|
||||
if (!systemQueuePut(head, message)) {
|
||||
message.next = null
|
||||
systemEnqueue(receiver, message)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@tailrec
|
||||
final def systemDrain(): SystemMessage = {
|
||||
final def systemDrain(newContents: SystemMessage): SystemMessage = {
|
||||
val head = systemQueueGet
|
||||
if (systemQueuePut(head, null)) SystemMessage.reverse(head) else systemDrain()
|
||||
if (systemQueuePut(head, newContents)) SystemMessage.reverse(head) else systemDrain(newContents)
|
||||
}
|
||||
|
||||
def hasSystemMessages: Boolean = systemQueueGet ne null
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -372,13 +394,11 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
|||
def pushTimeOut: Duration
|
||||
override def queue: BlockingQueue[Envelope]
|
||||
|
||||
def enqueue(receiver: ActorRef, handle: Envelope) {
|
||||
def enqueue(receiver: ActorRef, handle: Envelope): Unit =
|
||||
if (pushTimeOut.length > 0) {
|
||||
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
||||
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
|
||||
}
|
||||
if (!queue.offer(handle, pushTimeOut.length, pushTimeOut.unit))
|
||||
receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver)
|
||||
} else queue put handle
|
||||
}
|
||||
|
||||
def dequeue(): Envelope = queue.poll()
|
||||
}
|
||||
|
|
@ -410,18 +430,16 @@ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
|
|||
override def queue: BlockingDeque[Envelope]
|
||||
|
||||
def enqueue(receiver: ActorRef, handle: Envelope): Unit =
|
||||
if (pushTimeOut.length > 0)
|
||||
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
||||
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
|
||||
}
|
||||
else queue put handle
|
||||
if (pushTimeOut.length > 0) {
|
||||
if (!queue.offer(handle, pushTimeOut.length, pushTimeOut.unit))
|
||||
receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver)
|
||||
} else queue put handle
|
||||
|
||||
def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit =
|
||||
if (pushTimeOut.length > 0)
|
||||
queue.offerFirst(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
||||
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
|
||||
}
|
||||
else queue putFirst handle
|
||||
if (pushTimeOut.length > 0) {
|
||||
if (!queue.offerFirst(handle, pushTimeOut.length, pushTimeOut.unit))
|
||||
receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver)
|
||||
} else queue putFirst handle
|
||||
|
||||
def dequeue(): Envelope = queue.poll()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,30 +29,20 @@ object ThreadPoolConfig {
|
|||
val defaultTimeout: Duration = Duration(60000L, TimeUnit.MILLISECONDS)
|
||||
val defaultRejectionPolicy: RejectedExecutionHandler = new SaneRejectedExecutionHandler()
|
||||
|
||||
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int = {
|
||||
import scala.math.{ min, max }
|
||||
min(max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
|
||||
}
|
||||
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int =
|
||||
math.min(math.max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
|
||||
|
||||
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory =
|
||||
() ⇒ new ArrayBlockingQueue[Runnable](capacity, fair)
|
||||
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = () ⇒ new ArrayBlockingQueue[Runnable](capacity, fair)
|
||||
|
||||
def synchronousQueue(fair: Boolean): QueueFactory =
|
||||
() ⇒ new SynchronousQueue[Runnable](fair)
|
||||
def synchronousQueue(fair: Boolean): QueueFactory = () ⇒ new SynchronousQueue[Runnable](fair)
|
||||
|
||||
def linkedBlockingQueue(): QueueFactory =
|
||||
() ⇒ new LinkedBlockingQueue[Runnable]()
|
||||
def linkedBlockingQueue(): QueueFactory = () ⇒ new LinkedBlockingQueue[Runnable]()
|
||||
|
||||
def linkedBlockingQueue(capacity: Int): QueueFactory =
|
||||
() ⇒ new LinkedBlockingQueue[Runnable](capacity)
|
||||
def linkedBlockingQueue(capacity: Int): QueueFactory = () ⇒ new LinkedBlockingQueue[Runnable](capacity)
|
||||
|
||||
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory =
|
||||
() ⇒ queue
|
||||
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = () ⇒ queue
|
||||
|
||||
def reusableQueue(queueFactory: QueueFactory): QueueFactory = {
|
||||
val queue = queueFactory()
|
||||
() ⇒ queue
|
||||
}
|
||||
def reusableQueue(queueFactory: QueueFactory): QueueFactory = reusableQueue(queueFactory())
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -157,7 +147,8 @@ case class ThreadPoolConfigBuilder(config: ThreadPoolConfig) {
|
|||
def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
|
||||
this.copy(config = config.copy(queueFactory = newQueueFactory))
|
||||
|
||||
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder = fs.foldLeft(this)((c, f) ⇒ f.map(_(c)).getOrElse(c))
|
||||
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder =
|
||||
fs.foldLeft(this)((c, f) ⇒ f.map(_(c)).getOrElse(c))
|
||||
}
|
||||
|
||||
object MonitorableThreadFactory {
|
||||
|
|
|
|||
|
|
@ -1,19 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.event
|
||||
|
||||
import akka.actor._
|
||||
|
||||
/**
|
||||
* The contract of DeathWatch is not properly expressed using the type system
|
||||
* Whenever there is a publish, all listeners to the Terminated Actor should be atomically removed
|
||||
* A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down
|
||||
* See LocalDeathWatch for semantics
|
||||
*/
|
||||
abstract class DeathWatch extends ActorEventBus with ActorClassifier {
|
||||
type Event = Terminated
|
||||
|
||||
protected final def classify(event: Event): Classifier = event.actor
|
||||
}
|
||||
|
|
@ -4,12 +4,10 @@
|
|||
package akka.pattern
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.dispatch.{ Promise, Terminate, SystemMessage, Future }
|
||||
import akka.event.DeathWatch
|
||||
import akka.util.Timeout
|
||||
import annotation.tailrec
|
||||
import akka.util.Unsafe
|
||||
import akka.actor._
|
||||
import akka.dispatch._
|
||||
import akka.util.{ NonFatal, Timeout, Unsafe }
|
||||
|
||||
/**
|
||||
* This is what is used to complete a Future that is returned from an ask/? call,
|
||||
|
|
@ -164,6 +162,7 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
extends MinimalActorRef {
|
||||
import PromiseActorRef._
|
||||
import AbstractPromiseActorRef.stateOffset
|
||||
import AbstractPromiseActorRef.watchedByOffset
|
||||
|
||||
/**
|
||||
* As an optimization for the common (local) case we only register this PromiseActorRef
|
||||
|
|
@ -180,14 +179,43 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
@volatile
|
||||
private[this] var _stateDoNotCallMeDirectly: AnyRef = _
|
||||
|
||||
@inline
|
||||
private def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
|
||||
@volatile
|
||||
private[this] var _watchedByDoNotCallMeDirectly: Set[ActorRef] = ActorCell.emptyActorRefSet
|
||||
|
||||
@inline
|
||||
private def updateState(oldState: AnyRef, newState: AnyRef): Boolean = Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
|
||||
private[this] def watchedBy: Set[ActorRef] = Unsafe.instance.getObjectVolatile(this, watchedByOffset).asInstanceOf[Set[ActorRef]]
|
||||
|
||||
@inline
|
||||
private def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
|
||||
private[this] def updateWatchedBy(oldWatchedBy: Set[ActorRef], newWatchedBy: Set[ActorRef]): Boolean =
|
||||
Unsafe.instance.compareAndSwapObject(this, watchedByOffset, oldWatchedBy, newWatchedBy)
|
||||
|
||||
@tailrec // Returns false if the Promise is already completed
|
||||
private[this] final def addWatcher(watcher: ActorRef): Boolean = watchedBy match {
|
||||
case null ⇒ false
|
||||
case other ⇒ updateWatchedBy(other, other + watcher) || addWatcher(watcher)
|
||||
}
|
||||
|
||||
@tailrec
|
||||
private[this] final def remWatcher(watcher: ActorRef): Unit = watchedBy match {
|
||||
case null ⇒ ()
|
||||
case other ⇒ if (!updateWatchedBy(other, other - watcher)) remWatcher(watcher)
|
||||
}
|
||||
|
||||
@tailrec
|
||||
private[this] final def clearWatchers(): Set[ActorRef] = watchedBy match {
|
||||
case null ⇒ ActorCell.emptyActorRefSet
|
||||
case other ⇒ if (!updateWatchedBy(other, null)) clearWatchers() else other
|
||||
}
|
||||
|
||||
@inline
|
||||
private[this] def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
|
||||
|
||||
@inline
|
||||
private[this] def updateState(oldState: AnyRef, newState: AnyRef): Boolean =
|
||||
Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
|
||||
|
||||
@inline
|
||||
private[this] def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
|
||||
|
||||
override def getParent: InternalActorRef = provider.tempContainer
|
||||
|
||||
|
|
@ -218,20 +246,25 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = state match {
|
||||
case Stopped | _: StoppedWithPath ⇒ provider.deadLetters ! message
|
||||
case _ ⇒
|
||||
val completedJustNow = result.tryComplete {
|
||||
message match {
|
||||
case Status.Success(r) ⇒ Right(r)
|
||||
case Status.Failure(f) ⇒ Left(f)
|
||||
case other ⇒ Right(other)
|
||||
}
|
||||
case _ ⇒ if (!(result.tryComplete {
|
||||
message match {
|
||||
case Status.Success(r) ⇒ Right(r)
|
||||
case Status.Failure(f) ⇒ Left(f)
|
||||
case other ⇒ Right(other)
|
||||
}
|
||||
if (!completedJustNow) provider.deadLetters ! message
|
||||
})) provider.deadLetters ! message
|
||||
}
|
||||
|
||||
override def sendSystemMessage(message: SystemMessage): Unit = message match {
|
||||
case _: Terminate ⇒ stop()
|
||||
case _ ⇒
|
||||
case Watch(watchee, watcher) ⇒
|
||||
if (watchee == this && watcher != this) {
|
||||
if (!addWatcher(watcher)) watcher ! Terminated(watchee)(existenceConfirmed = true)
|
||||
} else System.err.println("BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, this))
|
||||
case Unwatch(watchee, watcher) ⇒
|
||||
if (watchee == this && watcher != this) remWatcher(watcher)
|
||||
else System.err.println("BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, this))
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
override def isTerminated: Boolean = state match {
|
||||
|
|
@ -241,23 +274,20 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
|
||||
@tailrec
|
||||
override def stop(): Unit = {
|
||||
def ensurePromiseCompleted(): Unit =
|
||||
def ensureCompleted(): Unit = {
|
||||
if (!result.isCompleted) result.tryComplete(Left(new ActorKilledException("Stopped")))
|
||||
val watchers = clearWatchers()
|
||||
if (!watchers.isEmpty) {
|
||||
val termination = Terminated(this)(existenceConfirmed = true)
|
||||
watchers foreach { w ⇒ try w.tell(termination, this) catch { case NonFatal(t) ⇒ /* FIXME LOG THIS */ } }
|
||||
}
|
||||
}
|
||||
state match {
|
||||
case null ⇒
|
||||
// if path was never queried nobody can possibly be watching us, so we don't have to publish termination either
|
||||
if (updateState(null, Stopped)) ensurePromiseCompleted()
|
||||
else stop()
|
||||
case null ⇒ // if path was never queried nobody can possibly be watching us, so we don't have to publish termination either
|
||||
if (updateState(null, Stopped)) ensureCompleted() else stop()
|
||||
case p: ActorPath ⇒
|
||||
if (updateState(p, StoppedWithPath(p))) {
|
||||
try {
|
||||
ensurePromiseCompleted()
|
||||
provider.deathWatch.publish(Terminated(this))
|
||||
} finally {
|
||||
provider.unregisterTempActor(p)
|
||||
}
|
||||
} else stop()
|
||||
case Stopped | _: StoppedWithPath ⇒
|
||||
if (updateState(p, StoppedWithPath(p))) { try ensureCompleted() finally provider.unregisterTempActor(p) } else stop()
|
||||
case Stopped | _: StoppedWithPath ⇒ // already stopped
|
||||
case Registering ⇒ stop() // spin until registration is completed before stopping
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,9 +4,9 @@
|
|||
|
||||
package akka.pattern
|
||||
|
||||
import akka.dispatch.{ Promise, Future }
|
||||
import akka.actor._
|
||||
import akka.util.{ Timeout, Duration }
|
||||
import akka.dispatch.{ Unwatch, Watch, Promise, Future }
|
||||
|
||||
trait GracefulStopSupport {
|
||||
/**
|
||||
|
|
@ -34,18 +34,21 @@ trait GracefulStopSupport {
|
|||
* is completed with failure [[akka.pattern.AskTimeoutException]].
|
||||
*/
|
||||
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = {
|
||||
if (target.isTerminated) {
|
||||
Promise.successful(true)
|
||||
} else system match {
|
||||
if (target.isTerminated) Promise.successful(true)
|
||||
else system match {
|
||||
case e: ExtendedActorSystem ⇒
|
||||
val internalTarget = target.asInstanceOf[InternalActorRef]
|
||||
val ref = PromiseActorRef(e.provider, Timeout(timeout))
|
||||
e.deathWatch.subscribe(ref, target)
|
||||
ref.result onComplete {
|
||||
case Right(Terminated(`target`)) ⇒ () // Ignore
|
||||
case _ ⇒ e.deathWatch.unsubscribe(ref, target)
|
||||
} // Just making sure we're not leaking here
|
||||
internalTarget.sendSystemMessage(Watch(target, ref))
|
||||
ref.result onComplete { // Just making sure we're not leaking here
|
||||
case Right(Terminated(`target`)) ⇒ ()
|
||||
case _ ⇒ internalTarget.sendSystemMessage(Unwatch(target, ref))
|
||||
}
|
||||
target ! PoisonPill
|
||||
ref.result map { case Terminated(`target`) ⇒ true }
|
||||
ref.result map {
|
||||
case Terminated(`target`) ⇒ true
|
||||
case _ ⇒ false
|
||||
}
|
||||
case s ⇒ throw new IllegalArgumentException("Unknown ActorSystem implementation: '" + s + "'")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,12 +29,6 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
|||
_supervisor,
|
||||
_path) {
|
||||
|
||||
// verify that a BalancingDispatcher is not used with a Router
|
||||
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher))
|
||||
throw new ConfigurationException(
|
||||
"Configuration for actor [" + _path.toString +
|
||||
"] is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
|
||||
|
||||
/*
|
||||
* CAUTION: RoutedActorRef is PROBLEMATIC
|
||||
* ======================================
|
||||
|
|
@ -47,14 +41,20 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
|||
* before we are done with them: lock the monitor of the actor cell (hence the
|
||||
* override of newActorCell) and use that to block the Router constructor for
|
||||
* as long as it takes to setup the RoutedActorRef itself.
|
||||
*
|
||||
* ===> I M P O R T A N T N O T I C E <===
|
||||
*
|
||||
* DO NOT THROW ANY EXCEPTIONS BEFORE THE FOLLOWING TRY-BLOCK WITHOUT
|
||||
* EXITING THE MONITOR OF THE actorCell!
|
||||
*
|
||||
* This is important, just don’t do it! No kidding.
|
||||
*/
|
||||
override def newActorCell(
|
||||
system: ActorSystemImpl,
|
||||
ref: InternalActorRef,
|
||||
props: Props,
|
||||
supervisor: InternalActorRef,
|
||||
receiveTimeout: Option[Duration]): ActorCell = {
|
||||
val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout)
|
||||
supervisor: InternalActorRef): ActorCell = {
|
||||
val cell = super.newActorCell(system, ref, props, supervisor)
|
||||
Unsafe.instance.monitorEnter(cell)
|
||||
cell
|
||||
}
|
||||
|
|
@ -74,6 +74,14 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
|||
|
||||
val route =
|
||||
try {
|
||||
// verify that a BalancingDispatcher is not used with a Router
|
||||
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) {
|
||||
actorContext.stop(actorContext.self)
|
||||
throw new ConfigurationException(
|
||||
"Configuration for actor [" + _path.toString +
|
||||
"] is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
|
||||
}
|
||||
|
||||
_routeeProvider = routerConfig.createRouteeProvider(actorContext)
|
||||
val r = routerConfig.createRoute(routeeProps, routeeProvider)
|
||||
// initial resize, before message send
|
||||
|
|
|
|||
|
|
@ -25,15 +25,14 @@ akka {
|
|||
periodic-tasks-initial-delay = 1s
|
||||
|
||||
# how often should the node send out gossip information?
|
||||
gossip-frequency = 1s
|
||||
gossip-interval = 1s
|
||||
|
||||
# how often should the leader perform maintenance tasks?
|
||||
leader-actions-frequency = 1s
|
||||
leader-actions-interval = 1s
|
||||
|
||||
# how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring?
|
||||
unreachable-nodes-reaper-frequency = 1s
|
||||
unreachable-nodes-reaper-interval = 1s
|
||||
|
||||
# accrual failure detection config
|
||||
failure-detector {
|
||||
|
||||
# defines the failure detector threshold
|
||||
|
|
@ -43,6 +42,8 @@ akka {
|
|||
# actual crashes
|
||||
threshold = 8
|
||||
|
||||
implementation-class = ""
|
||||
|
||||
max-sample-size = 1000
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,7 +4,8 @@
|
|||
|
||||
package akka.cluster
|
||||
|
||||
import akka.actor.{ ActorSystem, Address }
|
||||
import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
|
||||
import akka.remote.RemoteActorRefProvider
|
||||
import akka.event.Logging
|
||||
|
||||
import scala.collection.immutable.Map
|
||||
|
|
@ -23,11 +24,20 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
* Default threshold is 8, but can be configured in the Akka config.
|
||||
*/
|
||||
class AccrualFailureDetector(
|
||||
system: ActorSystem,
|
||||
address: Address,
|
||||
val system: ActorSystem,
|
||||
val threshold: Int = 8,
|
||||
val maxSampleSize: Int = 1000,
|
||||
val timeMachine: () ⇒ Long = System.currentTimeMillis) {
|
||||
val timeMachine: () ⇒ Long = System.currentTimeMillis) extends FailureDetector {
|
||||
|
||||
def this(
|
||||
system: ActorSystem,
|
||||
settings: ClusterSettings,
|
||||
timeMachine: () ⇒ Long = System.currentTimeMillis) =
|
||||
this(
|
||||
system,
|
||||
settings.FailureDetectorThreshold,
|
||||
settings.FailureDetectorMaxSampleSize,
|
||||
timeMachine)
|
||||
|
||||
private final val PhiFactor = 1.0 / math.log(10.0)
|
||||
|
||||
|
|
@ -36,7 +46,11 @@ class AccrualFailureDetector(
|
|||
/**
|
||||
* Holds the failure statistics for a specific node Address.
|
||||
*/
|
||||
private case class FailureStats(mean: Double = 0.0D, variance: Double = 0.0D, deviation: Double = 0.0D)
|
||||
private case class FailureStats(mean: Double = 0.0, variance: Double = 0.0, deviation: Double = 0.0)
|
||||
|
||||
// guess statistics for first heartbeat,
|
||||
// important so that connections with only one heartbeat becomes unavailble
|
||||
private val failureStatsFirstHeartbeat = FailureStats(mean = 1000.0)
|
||||
|
||||
/**
|
||||
* Implement using optimistic lockless concurrency, all state is represented
|
||||
|
|
@ -61,8 +75,8 @@ class AccrualFailureDetector(
|
|||
* Records a heartbeat for a connection.
|
||||
*/
|
||||
@tailrec
|
||||
final def heartbeat(connection: Address) {
|
||||
log.debug("Node [{}] - Heartbeat from connection [{}] ", address, connection)
|
||||
final def heartbeat(connection: Address): Unit = {
|
||||
log.debug("Heartbeat from connection [{}] ", connection)
|
||||
|
||||
val oldState = state.get
|
||||
val latestTimestamp = oldState.timestamps.get(connection)
|
||||
|
|
@ -72,7 +86,7 @@ class AccrualFailureDetector(
|
|||
// add starter records for this new connection
|
||||
val newState = oldState copy (
|
||||
version = oldState.version + 1,
|
||||
failureStats = oldState.failureStats + (connection -> FailureStats()),
|
||||
failureStats = oldState.failureStats + (connection -> failureStatsFirstHeartbeat),
|
||||
intervalHistory = oldState.intervalHistory + (connection -> IndexedSeq.empty[Long]),
|
||||
timestamps = oldState.timestamps + (connection -> timeMachine()),
|
||||
explicitRemovals = oldState.explicitRemovals - connection)
|
||||
|
|
@ -93,30 +107,24 @@ class AccrualFailureDetector(
|
|||
case _ ⇒ IndexedSeq.empty[Long]
|
||||
}) :+ interval
|
||||
|
||||
val newFailureStats =
|
||||
if (newIntervalsForConnection.size > 1) {
|
||||
val newFailureStats = {
|
||||
val newMean: Double = newIntervalsForConnection.sum.toDouble / newIntervalsForConnection.size
|
||||
|
||||
val newMean: Double = newIntervalsForConnection.sum / newIntervalsForConnection.size.toDouble
|
||||
|
||||
val oldConnectionFailureStats = oldState.failureStats.get(connection).getOrElse {
|
||||
throw new IllegalStateException("Can't calculate new failure statistics due to missing heartbeat history")
|
||||
}
|
||||
|
||||
val deviationSum =
|
||||
newIntervalsForConnection
|
||||
.map(_.toDouble)
|
||||
.foldLeft(0.0D)((x, y) ⇒ x + (y - newMean))
|
||||
|
||||
val newVariance: Double = deviationSum / newIntervalsForConnection.size.toDouble
|
||||
val newDeviation: Double = math.sqrt(newVariance)
|
||||
|
||||
val newFailureStats = oldConnectionFailureStats copy (mean = newMean, deviation = newDeviation, variance = newVariance)
|
||||
oldState.failureStats + (connection -> newFailureStats)
|
||||
|
||||
} else {
|
||||
oldState.failureStats
|
||||
val oldConnectionFailureStats = oldState.failureStats.get(connection).getOrElse {
|
||||
throw new IllegalStateException("Can't calculate new failure statistics due to missing heartbeat history")
|
||||
}
|
||||
|
||||
val deviationSum = (0.0d /: newIntervalsForConnection) { (mean, interval) ⇒
|
||||
mean + interval.toDouble - newMean
|
||||
}
|
||||
|
||||
val newVariance: Double = deviationSum / newIntervalsForConnection.size
|
||||
val newDeviation: Double = math.sqrt(newVariance)
|
||||
|
||||
val newFailureStats = oldConnectionFailureStats copy (mean = newMean, deviation = newDeviation, variance = newVariance)
|
||||
oldState.failureStats + (connection -> newFailureStats)
|
||||
}
|
||||
|
||||
val newState = oldState copy (version = oldState.version + 1,
|
||||
failureStats = newFailureStats,
|
||||
intervalHistory = oldState.intervalHistory + (connection -> newIntervalsForConnection),
|
||||
|
|
@ -132,8 +140,7 @@ class AccrualFailureDetector(
|
|||
* Calculates how likely it is that the connection has failed.
|
||||
* <p/>
|
||||
* If a connection does not have any records in failure detector then it is
|
||||
* considered dead. This is true either if the heartbeat have not started
|
||||
* yet or the connection have been explicitly removed.
|
||||
* considered healthy.
|
||||
* <p/>
|
||||
* Implementations of 'Cumulative Distribution Function' for Exponential Distribution.
|
||||
* For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597].
|
||||
|
|
@ -145,7 +152,7 @@ class AccrualFailureDetector(
|
|||
val phi =
|
||||
// if connection has been removed explicitly
|
||||
if (oldState.explicitRemovals.contains(connection)) Double.MaxValue
|
||||
else if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
|
||||
else if (oldTimestamp.isEmpty) 0.0 // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
|
||||
else {
|
||||
val timestampDiff = timeMachine() - oldTimestamp.get
|
||||
|
||||
|
|
@ -154,12 +161,11 @@ class AccrualFailureDetector(
|
|||
case _ ⇒ throw new IllegalStateException("Can't calculate Failure Detector Phi value for a node that have no heartbeat history")
|
||||
}
|
||||
|
||||
if (mean == 0.0D) 0.0D
|
||||
if (mean == 0.0) 0.0
|
||||
else PhiFactor * timestampDiff / mean
|
||||
}
|
||||
|
||||
// only log if PHI value is starting to get interesting
|
||||
if (phi > 0.0D) log.debug("Node [{}] - Phi value [{}] and threshold [{}] for connection [{}] ", address, phi, threshold, connection)
|
||||
log.debug("Phi value [{}] and threshold [{}] for connection [{}] ", phi, threshold, connection)
|
||||
phi
|
||||
}
|
||||
|
||||
|
|
@ -167,7 +173,8 @@ class AccrualFailureDetector(
|
|||
* Removes the heartbeat management for a connection.
|
||||
*/
|
||||
@tailrec
|
||||
final def remove(connection: Address) {
|
||||
final def remove(connection: Address): Unit = {
|
||||
log.debug("Remove connection [{}] ", connection)
|
||||
val oldState = state.get
|
||||
|
||||
if (oldState.failureStats.contains(connection)) {
|
||||
|
|
|
|||
|
|
@ -186,7 +186,7 @@ case class GossipOverview(
|
|||
*/
|
||||
case class Gossip(
|
||||
overview: GossipOverview = GossipOverview(),
|
||||
members: SortedSet[Member], // sorted set of members with their status, sorted by name
|
||||
members: SortedSet[Member], // sorted set of members with their status, sorted by address
|
||||
meta: Map[String, Array[Byte]] = Map.empty[String, Array[Byte]],
|
||||
version: VectorClock = VectorClock()) // vector clock version
|
||||
extends ClusterMessage // is a serializable cluster message
|
||||
|
|
@ -220,12 +220,8 @@ case class Gossip(
|
|||
// 1. merge vector clocks
|
||||
val mergedVClock = this.version merge that.version
|
||||
|
||||
// 2. group all members by Address => Vector[Member]
|
||||
var membersGroupedByAddress = Map.empty[Address, Vector[Member]]
|
||||
(this.members ++ that.members) foreach { m ⇒
|
||||
val ms = membersGroupedByAddress.get(m.address).getOrElse(Vector.empty[Member])
|
||||
membersGroupedByAddress += (m.address -> (ms :+ m))
|
||||
}
|
||||
// 2. group all members by Address => Seq[Member]
|
||||
val membersGroupedByAddress = (this.members.toSeq ++ that.members.toSeq).groupBy(_.address)
|
||||
|
||||
// 3. merge members by selecting the single Member with highest MemberStatus out of the Member groups
|
||||
val mergedMembers =
|
||||
|
|
@ -243,7 +239,7 @@ case class Gossip(
|
|||
this.overview.unreachable ++ that.overview.unreachable)
|
||||
|
||||
Gossip(mergedOverview, mergedMembers, mergedMeta, mergedVClock)
|
||||
}
|
||||
}
|
||||
|
||||
override def toString =
|
||||
"Gossip(" +
|
||||
|
|
@ -258,11 +254,9 @@ case class Gossip(
|
|||
* Manages routing of the different cluster commands.
|
||||
* Instantiated as a single instance for each Cluster - e.g. commands are serialized to Cluster message after message.
|
||||
*/
|
||||
final class ClusterCommandDaemon extends Actor {
|
||||
import ClusterUserAction._
|
||||
import ClusterLeaderAction._
|
||||
private[akka] final class ClusterCommandDaemon(cluster: Cluster) extends Actor {
|
||||
import ClusterAction._
|
||||
|
||||
val cluster = Cluster(context.system)
|
||||
val log = Logging(context.system, this)
|
||||
|
||||
def receive = {
|
||||
|
|
@ -280,9 +274,8 @@ final class ClusterCommandDaemon extends Actor {
|
|||
* Pooled and routed with N number of configurable instances.
|
||||
* Concurrent access to Cluster.
|
||||
*/
|
||||
final class ClusterGossipDaemon extends Actor {
|
||||
private[akka] final class ClusterGossipDaemon(cluster: Cluster) extends Actor {
|
||||
val log = Logging(context.system, this)
|
||||
val cluster = Cluster(context.system)
|
||||
|
||||
def receive = {
|
||||
case GossipEnvelope(sender, gossip) ⇒ cluster.receive(sender, gossip)
|
||||
|
|
@ -294,13 +287,13 @@ final class ClusterGossipDaemon extends Actor {
|
|||
/**
|
||||
* Supervisor managing the different Cluster daemons.
|
||||
*/
|
||||
final class ClusterDaemonSupervisor extends Actor {
|
||||
private[akka] final class ClusterDaemonSupervisor(cluster: Cluster) extends Actor {
|
||||
val log = Logging(context.system, this)
|
||||
val cluster = Cluster(context.system)
|
||||
|
||||
private val commands = context.actorOf(Props[ClusterCommandDaemon], "commands")
|
||||
private val commands = context.actorOf(Props(new ClusterCommandDaemon(cluster)), "commands")
|
||||
private val gossip = context.actorOf(
|
||||
Props[ClusterGossipDaemon].withRouter(RoundRobinRouter(cluster.clusterSettings.NrOfGossipDaemons)), "gossip")
|
||||
Props(new ClusterGossipDaemon(cluster)).withRouter(
|
||||
RoundRobinRouter(cluster.clusterSettings.NrOfGossipDaemons)), "gossip")
|
||||
|
||||
def receive = Actor.emptyBehavior
|
||||
|
||||
|
|
@ -319,7 +312,21 @@ object Cluster extends ExtensionId[Cluster] with ExtensionIdProvider {
|
|||
|
||||
override def lookup = Cluster
|
||||
|
||||
override def createExtension(system: ExtendedActorSystem): Cluster = new Cluster(system)
|
||||
override def createExtension(system: ExtendedActorSystem): Cluster = {
|
||||
val clusterSettings = new ClusterSettings(system.settings.config, system.name)
|
||||
|
||||
val failureDetector = clusterSettings.FailureDetectorImplementationClass match {
|
||||
case None ⇒ new AccrualFailureDetector(system, clusterSettings)
|
||||
case Some(fqcn) ⇒
|
||||
system.dynamicAccess.createInstanceFor[FailureDetector](
|
||||
fqcn, Seq((classOf[ActorSystem], system), (classOf[ClusterSettings], clusterSettings))) match {
|
||||
case Right(fd) ⇒ fd
|
||||
case Left(e) ⇒ throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString)
|
||||
}
|
||||
}
|
||||
|
||||
new Cluster(system, failureDetector)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -360,7 +367,7 @@ trait ClusterNodeMBean {
|
|||
* if (Cluster(system).isLeader) { ... }
|
||||
* }}}
|
||||
*/
|
||||
class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
||||
class Cluster(system: ExtendedActorSystem, val failureDetector: FailureDetector) extends Extension { clusterNode ⇒
|
||||
|
||||
/**
|
||||
* Represents the state for this Cluster. Implemented using optimistic lockless concurrency.
|
||||
|
|
@ -377,24 +384,15 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
|
||||
val remoteSettings = new RemoteSettings(system.settings.config, system.name)
|
||||
val clusterSettings = new ClusterSettings(system.settings.config, system.name)
|
||||
import clusterSettings._
|
||||
|
||||
val selfAddress = remote.transport.address
|
||||
val failureDetector = new AccrualFailureDetector(
|
||||
system, selfAddress, clusterSettings.FailureDetectorThreshold, clusterSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
private val vclockNode = VectorClock.Node(selfAddress.toString)
|
||||
|
||||
private val periodicTasksInitialDelay = clusterSettings.PeriodicTasksInitialDelay
|
||||
private val gossipFrequency = clusterSettings.GossipFrequency
|
||||
private val leaderActionsFrequency = clusterSettings.LeaderActionsFrequency
|
||||
private val unreachableNodesReaperFrequency = clusterSettings.UnreachableNodesReaperFrequency
|
||||
|
||||
implicit private val defaultTimeout = Timeout(remoteSettings.RemoteSystemDaemonAckTimeout)
|
||||
|
||||
private val autoDown = clusterSettings.AutoDown
|
||||
private val nrOfDeputyNodes = clusterSettings.NrOfDeputyNodes
|
||||
private val nrOfGossipDaemons = clusterSettings.NrOfGossipDaemons
|
||||
private val nodeToJoin: Option[Address] = clusterSettings.NodeToJoin filter (_ != selfAddress)
|
||||
private val nodeToJoin: Option[Address] = NodeToJoin filter (_ != selfAddress)
|
||||
|
||||
private val serialization = remote.serialization
|
||||
|
||||
|
|
@ -408,7 +406,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
|
||||
// create superisor for daemons under path "/system/cluster"
|
||||
private val clusterDaemons = {
|
||||
val createChild = CreateChild(Props[ClusterDaemonSupervisor], "cluster")
|
||||
val createChild = CreateChild(Props(new ClusterDaemonSupervisor(this)), "cluster")
|
||||
Await.result(system.systemGuardian ? createChild, defaultTimeout.duration) match {
|
||||
case a: ActorRef ⇒ a
|
||||
case e: Exception ⇒ throw e
|
||||
|
|
@ -429,17 +427,17 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
// ========================================================
|
||||
|
||||
// start periodic gossip to random nodes in cluster
|
||||
private val gossipCanceller = system.scheduler.schedule(periodicTasksInitialDelay, gossipFrequency) {
|
||||
private val gossipCanceller = system.scheduler.schedule(PeriodicTasksInitialDelay, GossipInterval) {
|
||||
gossip()
|
||||
}
|
||||
|
||||
// start periodic cluster failure detector reaping (moving nodes condemned by the failure detector to unreachable list)
|
||||
private val failureDetectorReaperCanceller = system.scheduler.schedule(periodicTasksInitialDelay, unreachableNodesReaperFrequency) {
|
||||
private val failureDetectorReaperCanceller = system.scheduler.schedule(PeriodicTasksInitialDelay, UnreachableNodesReaperInterval) {
|
||||
reapUnreachableMembers()
|
||||
}
|
||||
|
||||
// start periodic leader action management (only applies for the current leader)
|
||||
private val leaderActionsCanceller = system.scheduler.schedule(periodicTasksInitialDelay, leaderActionsFrequency) {
|
||||
private val leaderActionsCanceller = system.scheduler.schedule(PeriodicTasksInitialDelay, LeaderActionsInterval) {
|
||||
leaderActions()
|
||||
}
|
||||
|
||||
|
|
@ -493,7 +491,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
/**
|
||||
* Checks if we have a cluster convergence.
|
||||
*
|
||||
* @returns Some(convergedGossip) if convergence have been reached and None if not
|
||||
* @return Some(convergedGossip) if convergence have been reached and None if not
|
||||
*/
|
||||
def convergence: Option[Gossip] = convergence(latestGossip)
|
||||
|
||||
|
|
@ -594,27 +592,25 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
val localState = state.get
|
||||
val localGossip = localState.latestGossip
|
||||
val localMembers = localGossip.members
|
||||
val localOverview = localGossip.overview
|
||||
val localUnreachableMembers = localOverview.unreachable
|
||||
|
||||
// remove the node from the 'unreachable' set in case it is a DOWN node that is rejoining cluster
|
||||
val newUnreachableMembers = localUnreachableMembers filterNot { _.address == node }
|
||||
val newOverview = localOverview copy (unreachable = newUnreachableMembers)
|
||||
if (!localMembers.exists(_.address == node)) {
|
||||
|
||||
val newMembers = localMembers + Member(node, MemberStatus.Joining) // add joining node as Joining
|
||||
val newGossip = localGossip copy (overview = newOverview, members = newMembers)
|
||||
// remove the node from the 'unreachable' set in case it is a DOWN node that is rejoining cluster
|
||||
val newUnreachableMembers = localGossip.overview.unreachable filterNot { _.address == node }
|
||||
val newOverview = localGossip.overview copy (unreachable = newUnreachableMembers)
|
||||
|
||||
val versionedGossip = newGossip + vclockNode
|
||||
val seenVersionedGossip = versionedGossip seen selfAddress
|
||||
val newMembers = localMembers + Member(node, MemberStatus.Joining) // add joining node as Joining
|
||||
val newGossip = localGossip copy (overview = newOverview, members = newMembers)
|
||||
|
||||
val newState = localState copy (latestGossip = seenVersionedGossip)
|
||||
val versionedGossip = newGossip + vclockNode
|
||||
val seenVersionedGossip = versionedGossip seen selfAddress
|
||||
|
||||
if (!state.compareAndSet(localState, newState)) joining(node) // recur if we failed update
|
||||
else {
|
||||
if (node != selfAddress) failureDetector heartbeat node
|
||||
val newState = localState copy (latestGossip = seenVersionedGossip)
|
||||
|
||||
if (convergence(newState.latestGossip).isDefined) {
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newMembers }
|
||||
if (!state.compareAndSet(localState, newState)) joining(node) // recur if we failed update
|
||||
else {
|
||||
if (node != selfAddress) failureDetector heartbeat node
|
||||
notifyMembershipChangeListeners(localState, newState)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -640,13 +636,18 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
|
||||
if (!state.compareAndSet(localState, newState)) leaving(address) // recur if we failed update
|
||||
else {
|
||||
failureDetector heartbeat address // update heartbeat in failure detector
|
||||
if (convergence(newState.latestGossip).isDefined) {
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newMembers }
|
||||
}
|
||||
if (address != selfAddress) failureDetector heartbeat address // update heartbeat in failure detector
|
||||
notifyMembershipChangeListeners(localState, newState)
|
||||
}
|
||||
}
|
||||
|
||||
private def notifyMembershipChangeListeners(oldState: State, newState: State): Unit = {
|
||||
val oldMembersStatus = oldState.latestGossip.members.toSeq.map(m ⇒ (m.address, m.status))
|
||||
val newMembersStatus = newState.latestGossip.members.toSeq.map(m ⇒ (m.address, m.status))
|
||||
if (newMembersStatus != oldMembersStatus)
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newState.latestGossip.members }
|
||||
}
|
||||
|
||||
/**
|
||||
* State transition to EXITING.
|
||||
*/
|
||||
|
|
@ -722,9 +723,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
|
||||
if (!state.compareAndSet(localState, newState)) downing(address) // recur if we fail the update
|
||||
else {
|
||||
if (convergence(newState.latestGossip).isDefined) {
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newState.latestGossip.members }
|
||||
}
|
||||
notifyMembershipChangeListeners(localState, newState)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -739,7 +738,6 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
val winningGossip =
|
||||
if (remoteGossip.version <> localGossip.version) {
|
||||
// concurrent
|
||||
println("=======>>> CONCURRENT")
|
||||
val mergedGossip = remoteGossip merge localGossip
|
||||
val versionedMergedGossip = mergedGossip + vclockNode
|
||||
|
||||
|
|
@ -750,17 +748,14 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
versionedMergedGossip
|
||||
|
||||
} else if (remoteGossip.version < localGossip.version) {
|
||||
println("=======>>> LOCAL")
|
||||
// local gossip is newer
|
||||
localGossip
|
||||
|
||||
} else {
|
||||
println("=======>>> REMOTE")
|
||||
// remote gossip is newer
|
||||
remoteGossip
|
||||
}
|
||||
|
||||
println("=======>>> WINNING " + winningGossip.members.mkString(", "))
|
||||
val newState = localState copy (latestGossip = winningGossip seen selfAddress)
|
||||
|
||||
// if we won the race then update else try again
|
||||
|
|
@ -769,10 +764,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
log.info("Cluster Node [{}] - Receiving gossip from [{}]", selfAddress, sender.address)
|
||||
|
||||
if (sender.address != selfAddress) failureDetector heartbeat sender.address
|
||||
|
||||
if (convergence(newState.latestGossip).isDefined) {
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newState.latestGossip.members }
|
||||
}
|
||||
notifyMembershipChangeListeners(localState, newState)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -817,9 +809,11 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Gossips latest gossip to an address.
|
||||
*/
|
||||
private def gossipTo(address: Address): Unit = {
|
||||
private[akka] def gossipTo(address: Address): Unit = {
|
||||
val connection = clusterGossipConnectionFor(address)
|
||||
log.debug("Cluster Node [{}] - Gossiping to [{}]", selfAddress, connection)
|
||||
connection ! GossipEnvelope(self, latestGossip)
|
||||
|
|
@ -828,66 +822,90 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
/**
|
||||
* Gossips latest gossip to a random member in the set of members passed in as argument.
|
||||
*
|
||||
* @return 'true' if it gossiped to a "deputy" member.
|
||||
* @return the used [[akka.actor.Address] if any
|
||||
*/
|
||||
private def gossipToRandomNodeOf(addresses: Iterable[Address]): Boolean = {
|
||||
private def gossipToRandomNodeOf(addresses: IndexedSeq[Address]): Option[Address] = {
|
||||
log.debug("Cluster Node [{}] - Selecting random node to gossip to [{}]", selfAddress, addresses.mkString(", "))
|
||||
if (addresses.isEmpty) false
|
||||
else {
|
||||
val peers = addresses filter (_ != selfAddress) // filter out myself
|
||||
val peer = selectRandomNode(peers)
|
||||
gossipTo(peer)
|
||||
deputyNodes exists (peer == _)
|
||||
val peers = addresses filterNot (_ == selfAddress) // filter out myself
|
||||
val peer = selectRandomNode(peers)
|
||||
peer foreach gossipTo
|
||||
peer
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def gossipToUnreachableProbablity(membersSize: Int, unreachableSize: Int): Double =
|
||||
(membersSize + unreachableSize) match {
|
||||
case 0 ⇒ 0.0
|
||||
case sum ⇒ unreachableSize.toDouble / sum
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def gossipToDeputyProbablity(membersSize: Int, unreachableSize: Int, nrOfDeputyNodes: Int): Double = {
|
||||
if (nrOfDeputyNodes > membersSize) 1.0
|
||||
else if (nrOfDeputyNodes == 0) 0.0
|
||||
else (membersSize + unreachableSize) match {
|
||||
case 0 ⇒ 0.0
|
||||
case sum ⇒ (nrOfDeputyNodes + unreachableSize).toDouble / sum
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Initates a new round of gossip.
|
||||
*/
|
||||
private def gossip(): Unit = {
|
||||
private[akka] def gossip(): Unit = {
|
||||
val localState = state.get
|
||||
|
||||
log.debug("Cluster Node [{}] - Initiating new round of gossip", selfAddress)
|
||||
|
||||
if (isSingletonCluster(localState)) {
|
||||
// gossip to myself
|
||||
// TODO could perhaps be optimized, no need to gossip to myself when Up?
|
||||
gossipTo(selfAddress)
|
||||
|
||||
} else if (isAvailable(localState)) {
|
||||
log.debug("Cluster Node [{}] - Initiating new round of gossip", selfAddress)
|
||||
|
||||
val localGossip = localState.latestGossip
|
||||
val localMembers = localGossip.members
|
||||
// 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
|
||||
val localUnreachableMembers = localGossip.overview.unreachable.toIndexedSeq
|
||||
val localUnreachableSize = localUnreachableMembers.size
|
||||
|
||||
// 1. gossip to alive members
|
||||
val gossipedToDeputy = gossipToRandomNodeOf(localMembers map { _.address })
|
||||
val gossipedToAlive = gossipToRandomNodeOf(localMemberAddresses)
|
||||
|
||||
// 2. gossip to unreachable members
|
||||
if (localUnreachableSize > 0) {
|
||||
val probability: Double = localUnreachableSize / (localMembersSize + 1)
|
||||
if (ThreadLocalRandom.current.nextDouble() < probability) gossipToRandomNodeOf(localUnreachableMembers.map(_.address))
|
||||
val probability = gossipToUnreachableProbablity(localMembersSize, localUnreachableSize)
|
||||
if (ThreadLocalRandom.current.nextDouble() < probability)
|
||||
gossipToRandomNodeOf(localUnreachableMembers.map(_.address))
|
||||
}
|
||||
|
||||
// 3. gossip to a deputy nodes for facilitating partition healing
|
||||
val deputies = deputyNodes
|
||||
if ((!gossipedToDeputy || localMembersSize < 1) && deputies.nonEmpty) {
|
||||
if (localMembersSize == 0) gossipToRandomNodeOf(deputies)
|
||||
else {
|
||||
val probability = 1.0 / localMembersSize + localUnreachableSize
|
||||
if (ThreadLocalRandom.current.nextDouble() <= probability) gossipToRandomNodeOf(deputies)
|
||||
}
|
||||
val deputies = deputyNodes(localMemberAddresses)
|
||||
val alreadyGossipedToDeputy = gossipedToAlive.map(deputies.contains(_)).getOrElse(false)
|
||||
if ((!alreadyGossipedToDeputy || localMembersSize < NrOfDeputyNodes) && deputies.nonEmpty) {
|
||||
val probability = gossipToDeputyProbablity(localMembersSize, localUnreachableSize, NrOfDeputyNodes)
|
||||
if (ThreadLocalRandom.current.nextDouble() < probability)
|
||||
gossipToRandomNodeOf(deputies)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Reaps the unreachable members (moves them to the 'unreachable' list in the cluster overview) according to the failure detector's verdict.
|
||||
*/
|
||||
@tailrec
|
||||
final private def reapUnreachableMembers(): Unit = {
|
||||
final private[akka] def reapUnreachableMembers(): Unit = {
|
||||
val localState = state.get
|
||||
|
||||
if (!isSingletonCluster(localState) && isAvailable(localState)) {
|
||||
|
|
@ -895,7 +913,6 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
|
||||
val localGossip = localState.latestGossip
|
||||
val localOverview = localGossip.overview
|
||||
val localSeen = localOverview.seen
|
||||
val localMembers = localGossip.members
|
||||
val localUnreachableMembers = localGossip.overview.unreachable
|
||||
|
||||
|
|
@ -920,19 +937,19 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
else {
|
||||
log.info("Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]", selfAddress, newlyDetectedUnreachableMembers.mkString(", "))
|
||||
|
||||
if (convergence(newState.latestGossip).isDefined) {
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newMembers }
|
||||
}
|
||||
notifyMembershipChangeListeners(localState, newState)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Runs periodic leader actions, such as auto-downing unreachable nodes, assigning partitions etc.
|
||||
*/
|
||||
@tailrec
|
||||
final private def leaderActions(): Unit = {
|
||||
final private[akka] def leaderActions(): Unit = {
|
||||
val localState = state.get
|
||||
val localGossip = localState.latestGossip
|
||||
val localMembers = localGossip.members
|
||||
|
|
@ -1003,7 +1020,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
|
||||
localGossip copy (members = newMembers) // update gossip
|
||||
|
||||
} else if (autoDown) {
|
||||
} 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
|
||||
|
||||
|
|
@ -1044,9 +1061,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
// if we won the race then update else try again
|
||||
if (!state.compareAndSet(localState, newState)) leaderActions() // recur
|
||||
else {
|
||||
if (convergence(newState.latestGossip).isDefined) {
|
||||
newState.memberMembershipChangeListeners foreach { _ notify newGossip.members }
|
||||
}
|
||||
notifyMembershipChangeListeners(localState, newState)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1074,10 +1089,8 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
val seen = gossip.overview.seen
|
||||
val views = Set.empty[VectorClock] ++ seen.values
|
||||
|
||||
println("=======>>> VIEWS " + views.size)
|
||||
if (views.size == 1) {
|
||||
log.debug("Cluster Node [{}] - Cluster convergence reached", selfAddress)
|
||||
println("=======>>> ----------------------- HAS CONVERGENCE")
|
||||
log.debug("Cluster Node [{}] - Cluster convergence reached: [{}]", selfAddress, gossip.members.mkString(", "))
|
||||
Some(gossip)
|
||||
} else None
|
||||
} else None
|
||||
|
|
@ -1111,11 +1124,17 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒
|
|||
private def clusterGossipConnectionFor(address: Address): ActorRef = system.actorFor(RootActorPath(address) / "system" / "cluster" / "gossip")
|
||||
|
||||
/**
|
||||
* Gets an Iterable with the addresses of a all the 'deputy' nodes - excluding this node if part of the group.
|
||||
* Gets the addresses of a all the 'deputy' nodes - excluding this node if part of the group.
|
||||
*/
|
||||
private def deputyNodes: Iterable[Address] = state.get.latestGossip.members.toIterable map (_.address) drop 1 take nrOfDeputyNodes filter (_ != selfAddress)
|
||||
private def deputyNodes(addresses: IndexedSeq[Address]): IndexedSeq[Address] =
|
||||
addresses drop 1 take NrOfDeputyNodes filterNot (_ == selfAddress)
|
||||
|
||||
private def selectRandomNode(addresses: Iterable[Address]): Address = addresses.toSeq(ThreadLocalRandom.current nextInt addresses.size)
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] =
|
||||
if (addresses.isEmpty) None
|
||||
else Some(addresses(ThreadLocalRandom.current nextInt addresses.size))
|
||||
|
||||
private def isSingletonCluster(currentState: State): Boolean = currentState.latestGossip.members.size == 1
|
||||
|
||||
|
|
|
|||
|
|
@ -15,14 +15,18 @@ class ClusterSettings(val config: Config, val systemName: String) {
|
|||
import config._
|
||||
val FailureDetectorThreshold = getInt("akka.cluster.failure-detector.threshold")
|
||||
val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size")
|
||||
val FailureDetectorImplementationClass: Option[String] = getString("akka.cluster.failure-detector.implementation-class") match {
|
||||
case "" ⇒ None
|
||||
case fqcn ⇒ Some(fqcn)
|
||||
}
|
||||
val NodeToJoin: Option[Address] = getString("akka.cluster.node-to-join") match {
|
||||
case "" ⇒ None
|
||||
case AddressFromURIString(addr) ⇒ Some(addr)
|
||||
}
|
||||
val PeriodicTasksInitialDelay = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
|
||||
val GossipFrequency = Duration(getMilliseconds("akka.cluster.gossip-frequency"), MILLISECONDS)
|
||||
val LeaderActionsFrequency = Duration(getMilliseconds("akka.cluster.leader-actions-frequency"), MILLISECONDS)
|
||||
val UnreachableNodesReaperFrequency = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-frequency"), MILLISECONDS)
|
||||
val GossipInterval = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
|
||||
val LeaderActionsInterval = Duration(getMilliseconds("akka.cluster.leader-actions-interval"), MILLISECONDS)
|
||||
val UnreachableNodesReaperInterval = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS)
|
||||
val NrOfGossipDaemons = getInt("akka.cluster.nr-of-gossip-daemons")
|
||||
val NrOfDeputyNodes = getInt("akka.cluster.nr-of-deputy-nodes")
|
||||
val AutoDown = getBoolean("akka.cluster.auto-down")
|
||||
|
|
|
|||
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import akka.actor.Address
|
||||
|
||||
/**
|
||||
* Interface for Akka failure detectors.
|
||||
*/
|
||||
trait FailureDetector {
|
||||
|
||||
/**
|
||||
* Returns true if the connection is considered to be up and healthy and returns false otherwise.
|
||||
*/
|
||||
def isAvailable(connection: Address): Boolean
|
||||
|
||||
/**
|
||||
* Records a heartbeat for a connection.
|
||||
*/
|
||||
def heartbeat(connection: Address): Unit
|
||||
|
||||
/**
|
||||
* Removes the heartbeat management for a connection.
|
||||
*/
|
||||
def remove(connection: Address): Unit
|
||||
}
|
||||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
|
|
@ -19,32 +18,32 @@ object ClientDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
|
|||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class ClientDowningNodeThatIsUnreachableMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec
|
||||
class ClientDowningNodeThatIsUnreachableMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec
|
||||
class ClientDowningNodeThatIsUnreachableMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec
|
||||
class ClientDowningNodeThatIsUnreachableMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec
|
||||
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
class ClientDowningNodeThatIsUnreachableSpec
|
||||
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class ClientDowningNodeThatIsUnreachableSpec
|
||||
extends MultiNodeSpec(ClientDowningNodeThatIsUnreachableMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with BeforeAndAfter {
|
||||
import ClientDowningNodeThatIsUnreachableMultiJvmSpec._
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
override def initialParticipants = 4
|
||||
import ClientDowningNodeThatIsUnreachableMultiJvmSpec._
|
||||
|
||||
"Client of a 4 node cluster" must {
|
||||
|
||||
"be able to DOWN a node that is UNREACHABLE (killed)" taggedAs LongRunningTest in {
|
||||
val thirdAddress = node(third).address
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
|
||||
val thirdAddress = node(third).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
// kill 'third' node
|
||||
testConductor.shutdown(third, 0)
|
||||
testConductor.removeNode(third)
|
||||
markNodeAsUnavailable(thirdAddress)
|
||||
|
||||
// mark 'third' node as DOWN
|
||||
cluster.down(thirdAddress)
|
||||
|
|
@ -52,28 +51,19 @@ class ClientDowningNodeThatIsUnreachableSpec
|
|||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false)
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(node(first).address)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
testConductor.enter("all-up")
|
||||
testConductor.enter("down-third-node")
|
||||
}
|
||||
|
||||
runOn(second, fourth) {
|
||||
cluster.join(node(first).address)
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
|
||||
val thirdAddress = node(third).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
testConductor.enter("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
|
|
@ -19,58 +18,50 @@ object ClientDowningNodeThatIsUpMultiJvmSpec extends MultiNodeConfig {
|
|||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class ClientDowningNodeThatIsUpMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec
|
||||
class ClientDowningNodeThatIsUpMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec
|
||||
class ClientDowningNodeThatIsUpMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec
|
||||
class ClientDowningNodeThatIsUpMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec
|
||||
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
|
||||
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
|
||||
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
|
||||
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
class ClientDowningNodeThatIsUpSpec
|
||||
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
|
||||
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
|
||||
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
|
||||
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class ClientDowningNodeThatIsUpSpec
|
||||
extends MultiNodeSpec(ClientDowningNodeThatIsUpMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with BeforeAndAfter {
|
||||
import ClientDowningNodeThatIsUpMultiJvmSpec._
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
override def initialParticipants = 4
|
||||
import ClientDowningNodeThatIsUpMultiJvmSpec._
|
||||
|
||||
"Client of a 4 node cluster" must {
|
||||
|
||||
"be able to DOWN a node that is UP (healthy and available)" taggedAs LongRunningTest in {
|
||||
val thirdAddress = node(third).address
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
|
||||
val thirdAddress = node(third).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
// mark 'third' node as DOWN
|
||||
cluster.down(thirdAddress)
|
||||
testConductor.enter("down-third-node")
|
||||
|
||||
markNodeAsUnavailable(thirdAddress)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false)
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(node(first).address)
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
testConductor.enter("all-up")
|
||||
testConductor.enter("down-third-node")
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
runOn(second, fourth) {
|
||||
cluster.join(node(first).address)
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
|
||||
val thirdAddress = node(third).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
testConductor.enter("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,122 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
import akka.actor.Address
|
||||
|
||||
object ConvergenceMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold = 4")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class ConvergenceWithFailureDetectorPuppetMultiJvmNode1 extends ConvergenceSpec with FailureDetectorPuppetStrategy
|
||||
class ConvergenceWithFailureDetectorPuppetMultiJvmNode2 extends ConvergenceSpec with FailureDetectorPuppetStrategy
|
||||
class ConvergenceWithFailureDetectorPuppetMultiJvmNode3 extends ConvergenceSpec with FailureDetectorPuppetStrategy
|
||||
class ConvergenceWithFailureDetectorPuppetMultiJvmNode4 extends ConvergenceSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
class ConvergenceWithAccrualFailureDetectorMultiJvmNode1 extends ConvergenceSpec with AccrualFailureDetectorStrategy
|
||||
class ConvergenceWithAccrualFailureDetectorMultiJvmNode2 extends ConvergenceSpec with AccrualFailureDetectorStrategy
|
||||
class ConvergenceWithAccrualFailureDetectorMultiJvmNode3 extends ConvergenceSpec with AccrualFailureDetectorStrategy
|
||||
class ConvergenceWithAccrualFailureDetectorMultiJvmNode4 extends ConvergenceSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class ConvergenceSpec
|
||||
extends MultiNodeSpec(ConvergenceMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import ConvergenceMultiJvmSpec._
|
||||
|
||||
"A cluster of 3 members" must {
|
||||
|
||||
"reach initial convergence" taggedAs LongRunningTest ignore {
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(fourth) {
|
||||
// doesn't join immediately
|
||||
}
|
||||
|
||||
testConductor.enter("after-1")
|
||||
}
|
||||
|
||||
"not reach convergence while any nodes are unreachable" taggedAs LongRunningTest ignore {
|
||||
val thirdAddress = node(third).address
|
||||
testConductor.enter("before-shutdown")
|
||||
|
||||
runOn(first) {
|
||||
// kill 'third' node
|
||||
testConductor.shutdown(third, 0)
|
||||
markNodeAsUnavailable(thirdAddress)
|
||||
}
|
||||
|
||||
runOn(first, second) {
|
||||
val firstAddress = node(first).address
|
||||
val secondAddress = node(second).address
|
||||
|
||||
within(28 seconds) {
|
||||
// third becomes unreachable
|
||||
awaitCond(cluster.latestGossip.overview.unreachable.size == 1)
|
||||
awaitCond(cluster.latestGossip.members.size == 2)
|
||||
awaitCond(cluster.latestGossip.members.forall(_.status == MemberStatus.Up))
|
||||
awaitSeenSameState(Seq(firstAddress, secondAddress))
|
||||
// still one unreachable
|
||||
cluster.latestGossip.overview.unreachable.size must be(1)
|
||||
cluster.latestGossip.overview.unreachable.head.address must be(thirdAddress)
|
||||
// and therefore no convergence
|
||||
cluster.convergence.isDefined must be(false)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
}
|
||||
|
||||
"not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest ignore {
|
||||
runOn(fourth) {
|
||||
// try to join
|
||||
cluster.join(node(first).address)
|
||||
}
|
||||
|
||||
val firstAddress = node(first).address
|
||||
val secondAddress = node(second).address
|
||||
val fourthAddress = node(fourth).address
|
||||
|
||||
def memberStatus(address: Address): Option[MemberStatus] =
|
||||
cluster.latestGossip.members.collectFirst { case m if m.address == address ⇒ m.status }
|
||||
|
||||
def assertNotMovedUp: Unit = {
|
||||
within(20 seconds) {
|
||||
awaitCond(cluster.latestGossip.members.size == 3)
|
||||
awaitSeenSameState(Seq(firstAddress, secondAddress, fourthAddress))
|
||||
memberStatus(firstAddress) must be(Some(MemberStatus.Up))
|
||||
memberStatus(secondAddress) must be(Some(MemberStatus.Up))
|
||||
// leader is not allowed to move the new node to Up
|
||||
memberStatus(fourthAddress) must be(Some(MemberStatus.Joining))
|
||||
// still no convergence
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
}
|
||||
|
||||
runOn(first, second, fourth) {
|
||||
for (n ← 1 to 5) {
|
||||
log.debug("assertNotMovedUp#" + n)
|
||||
assertNotMovedUp
|
||||
// wait and then check again
|
||||
1.second.dilated.sleep
|
||||
}
|
||||
}
|
||||
|
||||
testConductor.enter("after-3")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,61 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import akka.actor.Address
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
|
||||
/**
|
||||
* Base trait for all failure detector strategies.
|
||||
*/
|
||||
trait FailureDetectorStrategy {
|
||||
|
||||
/**
|
||||
* Get or create the FailureDetector to be used in the cluster node.
|
||||
* To be defined by subclass.
|
||||
*/
|
||||
def failureDetector: FailureDetector
|
||||
|
||||
/**
|
||||
* Marks a node as available in the failure detector.
|
||||
* To be defined by subclass.
|
||||
*/
|
||||
def markNodeAsAvailable(address: Address): Unit
|
||||
|
||||
/**
|
||||
* Marks a node as unavailable in the failure detector.
|
||||
* To be defined by subclass.
|
||||
*/
|
||||
def markNodeAsUnavailable(address: Address): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines a FailureDetectorPuppet-based FailureDetectorStrategy.
|
||||
*/
|
||||
trait FailureDetectorPuppetStrategy extends FailureDetectorStrategy { self: MultiNodeSpec ⇒
|
||||
|
||||
/**
|
||||
* The puppet instance. Separated from 'failureDetector' field so we don't have to cast when using the puppet specific methods.
|
||||
*/
|
||||
private val puppet = new FailureDetectorPuppet(system)
|
||||
|
||||
override def failureDetector: FailureDetector = puppet
|
||||
|
||||
override def markNodeAsAvailable(address: Address): Unit = puppet markNodeAsAvailable address
|
||||
|
||||
override def markNodeAsUnavailable(address: Address): Unit = puppet markNodeAsUnavailable address
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines a AccrualFailureDetector-based FailureDetectorStrategy.
|
||||
*/
|
||||
trait AccrualFailureDetectorStrategy extends FailureDetectorStrategy { self: MultiNodeSpec ⇒
|
||||
|
||||
override val failureDetector: FailureDetector = new AccrualFailureDetector(system, new ClusterSettings(system.settings.config, system.name))
|
||||
|
||||
override def markNodeAsAvailable(address: Address): Unit = { /* no-op */ }
|
||||
|
||||
override def markNodeAsUnavailable(address: Address): Unit = { /* no-op */ }
|
||||
}
|
||||
|
|
@ -3,7 +3,6 @@
|
|||
*/
|
||||
package akka.cluster
|
||||
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
|
|
@ -16,49 +15,40 @@ object GossipingAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig {
|
|||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold=4")).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold = 4")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class GossipingAccrualFailureDetectorMultiJvmNode1 extends GossipingAccrualFailureDetectorSpec
|
||||
class GossipingAccrualFailureDetectorMultiJvmNode2 extends GossipingAccrualFailureDetectorSpec
|
||||
class GossipingAccrualFailureDetectorMultiJvmNode3 extends GossipingAccrualFailureDetectorSpec
|
||||
class GossipingWithAccrualFailureDetectorMultiJvmNode1 extends GossipingAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
|
||||
class GossipingWithAccrualFailureDetectorMultiJvmNode2 extends GossipingAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
|
||||
class GossipingWithAccrualFailureDetectorMultiJvmNode3 extends GossipingAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class GossipingAccrualFailureDetectorSpec
|
||||
extends MultiNodeSpec(GossipingAccrualFailureDetectorMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class GossipingAccrualFailureDetectorSpec extends MultiNodeSpec(GossipingAccrualFailureDetectorMultiJvmSpec)
|
||||
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import GossipingAccrualFailureDetectorMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
after {
|
||||
testConductor.enter("after")
|
||||
}
|
||||
|
||||
"A Gossip-driven Failure Detector" must {
|
||||
|
||||
"receive gossip heartbeats so that all member nodes in the cluster are marked 'available'" taggedAs LongRunningTest in {
|
||||
// make sure that the node-to-join is started before other join
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
cluster.join(firstAddress)
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
5.seconds.dilated.sleep // let them gossip
|
||||
cluster.failureDetector.isAvailable(firstAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(secondAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(thirdAddress) must be(true)
|
||||
|
||||
testConductor.enter("after-1")
|
||||
}
|
||||
|
||||
"mark node as 'unavailable' if a node in the cluster is shut down (and its heartbeats stops)" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
testConductor.shutdown(third, 0)
|
||||
testConductor.removeNode(third)
|
||||
}
|
||||
|
||||
runOn(first, second) {
|
||||
|
|
@ -68,7 +58,8 @@ abstract class GossipingAccrualFailureDetectorSpec extends MultiNodeSpec(Gossipi
|
|||
cluster.failureDetector.isAvailable(firstAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(secondAddress) must be(true)
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,21 +17,22 @@ object JoinTwoClustersMultiJvmSpec extends MultiNodeConfig {
|
|||
val c1 = role("c1")
|
||||
val c2 = role("c2")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
commonConfig(debugConfig(on = true).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec
|
||||
class JoinTwoClustersMultiJvmNode2 extends JoinTwoClustersSpec
|
||||
class JoinTwoClustersMultiJvmNode3 extends JoinTwoClustersSpec
|
||||
class JoinTwoClustersMultiJvmNode4 extends JoinTwoClustersSpec
|
||||
class JoinTwoClustersMultiJvmNode5 extends JoinTwoClustersSpec
|
||||
class JoinTwoClustersMultiJvmNode6 extends JoinTwoClustersSpec
|
||||
class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
class JoinTwoClustersMultiJvmNode2 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
class JoinTwoClustersMultiJvmNode3 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
class JoinTwoClustersMultiJvmNode4 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
class JoinTwoClustersMultiJvmNode5 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
class JoinTwoClustersMultiJvmNode6 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class JoinTwoClustersSpec
|
||||
extends MultiNodeSpec(JoinTwoClustersMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender {
|
||||
import JoinTwoClustersMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 6
|
||||
|
||||
lazy val a1Address = node(a1).address
|
||||
lazy val b1Address = node(b1).address
|
||||
lazy val c1Address = node(c1).address
|
||||
|
|
@ -41,7 +42,7 @@ abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvm
|
|||
"be able to 'elect' a single leader after joining (A -> B)" taggedAs LongRunningTest in {
|
||||
// make sure that the node-to-join is started before other join
|
||||
runOn(a1, b1, c1) {
|
||||
cluster.self
|
||||
startClusterNode()
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
|
|
@ -75,7 +76,6 @@ abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvm
|
|||
assertLeader(c1, c2)
|
||||
|
||||
testConductor.enter("four-members")
|
||||
|
||||
}
|
||||
|
||||
"be able to 'elect' a single leader after joining (C -> A + B)" taggedAs LongRunningTest in {
|
||||
|
|
@ -91,5 +91,4 @@ abstract class JoinTwoClustersSpec extends MultiNodeSpec(JoinTwoClustersMultiJvm
|
|||
testConductor.enter("six-members")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,11 +4,10 @@
|
|||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.actor.Address
|
||||
import akka.actor._
|
||||
import akka.util.duration._
|
||||
|
||||
object LeaderDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
|
||||
|
|
@ -17,109 +16,87 @@ object LeaderDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
|
|||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = true).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
auto-down = on
|
||||
failure-detector.threshold = 4
|
||||
}
|
||||
""")).
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.auto-down = on")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class LeaderDowningNodeThatIsUnreachableMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec
|
||||
class LeaderDowningNodeThatIsUnreachableMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec
|
||||
class LeaderDowningNodeThatIsUnreachableMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec
|
||||
class LeaderDowningNodeThatIsUnreachableMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec
|
||||
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
class LeaderDowningNodeThatIsUnreachableSpec
|
||||
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class LeaderDowningNodeThatIsUnreachableSpec
|
||||
extends MultiNodeSpec(LeaderDowningNodeThatIsUnreachableMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with BeforeAndAfter {
|
||||
import LeaderDowningNodeThatIsUnreachableMultiJvmSpec._
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
override def initialParticipants = 4
|
||||
import LeaderDowningNodeThatIsUnreachableMultiJvmSpec._
|
||||
|
||||
"The Leader in a 4 node cluster" must {
|
||||
|
||||
"be able to DOWN a 'last' node that is UNREACHABLE" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
val fourthAddress = node(fourth).address
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
|
||||
val fourthAddress = node(fourth).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
// kill 'fourth' node
|
||||
testConductor.shutdown(fourth, 0)
|
||||
testConductor.removeNode(fourth)
|
||||
testConductor.enter("down-fourth-node")
|
||||
|
||||
// mark the node as unreachable in the failure detector
|
||||
markNodeAsUnavailable(fourthAddress)
|
||||
|
||||
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
cluster.join(node(first).address)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
testConductor.enter("all-up")
|
||||
testConductor.enter("down-fourth-node")
|
||||
}
|
||||
|
||||
runOn(second, third) {
|
||||
cluster.join(node(first).address)
|
||||
awaitUpConvergence(numberOfMembers = 4)
|
||||
|
||||
val fourthAddress = node(fourth).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
testConductor.enter("down-fourth-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion-1")
|
||||
}
|
||||
|
||||
"be able to DOWN a 'middle' node that is UNREACHABLE" taggedAs LongRunningTest in {
|
||||
val secondAddress = node(second).address
|
||||
|
||||
testConductor.enter("before-down-second-node")
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
|
||||
val secondAddress = node(second).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
// kill 'second' node
|
||||
testConductor.shutdown(second, 0)
|
||||
testConductor.removeNode(second)
|
||||
testConductor.enter("down-second-node")
|
||||
|
||||
// mark the node as unreachable in the failure detector
|
||||
markNodeAsUnavailable(secondAddress)
|
||||
|
||||
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(node(first).address)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
testConductor.enter("all-up")
|
||||
testConductor.enter("down-second-node")
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(node(first).address)
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
|
||||
val secondAddress = node(second).address
|
||||
testConductor.enter("all-up")
|
||||
|
||||
testConductor.enter("down-second-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds)
|
||||
testConductor.enter("await-completion")
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,45 +17,44 @@ object LeaderElectionMultiJvmSpec extends MultiNodeConfig {
|
|||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class LeaderElectionMultiJvmNode1 extends LeaderElectionSpec
|
||||
class LeaderElectionMultiJvmNode2 extends LeaderElectionSpec
|
||||
class LeaderElectionMultiJvmNode3 extends LeaderElectionSpec
|
||||
class LeaderElectionMultiJvmNode4 extends LeaderElectionSpec
|
||||
class LeaderElectionMultiJvmNode5 extends LeaderElectionSpec
|
||||
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode1 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode2 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode3 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode4 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderElectionWithFailureDetectorPuppetMultiJvmNode5 extends LeaderElectionSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode1 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode2 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode3 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode4 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
|
||||
class LeaderElectionWithAccrualFailureDetectorMultiJvmNode5 extends LeaderElectionSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class LeaderElectionSpec
|
||||
extends MultiNodeSpec(LeaderElectionMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class LeaderElectionSpec extends MultiNodeSpec(LeaderElectionMultiJvmSpec) with MultiNodeClusterSpec {
|
||||
import LeaderElectionMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 5
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
|
||||
// sorted in the order used by the cluster
|
||||
lazy val roles = Seq(first, second, third, fourth).sorted
|
||||
lazy val sortedRoles = Seq(first, second, third, fourth).sorted
|
||||
|
||||
"A cluster of four nodes" must {
|
||||
|
||||
"be able to 'elect' a single leader" taggedAs LongRunningTest in {
|
||||
// make sure that the node-to-join is started before other join
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
if (myself != controller) {
|
||||
cluster.join(firstAddress)
|
||||
awaitUpConvergence(numberOfMembers = roles.size)
|
||||
cluster.isLeader must be(myself == roles.head)
|
||||
assertLeaderIn(roles)
|
||||
cluster.isLeader must be(myself == sortedRoles.head)
|
||||
assertLeaderIn(sortedRoles)
|
||||
}
|
||||
|
||||
testConductor.enter("after")
|
||||
}
|
||||
|
||||
def shutdownLeaderAndVerifyNewLeader(alreadyShutdown: Int): Unit = {
|
||||
val currentRoles = roles.drop(alreadyShutdown)
|
||||
val currentRoles = sortedRoles.drop(alreadyShutdown)
|
||||
currentRoles.size must be >= (2)
|
||||
val leader = currentRoles.head
|
||||
val aUser = currentRoles.last
|
||||
|
|
@ -64,13 +63,14 @@ abstract class LeaderElectionSpec extends MultiNodeSpec(LeaderElectionMultiJvmSp
|
|||
myself match {
|
||||
|
||||
case `controller` ⇒
|
||||
val leaderAddress = node(leader).address
|
||||
testConductor.enter("before-shutdown")
|
||||
testConductor.shutdown(leader, 0)
|
||||
testConductor.removeNode(leader)
|
||||
testConductor.enter("after-shutdown", "after-down", "completed")
|
||||
markNodeAsUnavailable(leaderAddress)
|
||||
|
||||
case `leader` ⇒
|
||||
testConductor.enter("before-shutdown")
|
||||
testConductor.enter("before-shutdown", "after-shutdown")
|
||||
// this node will be shutdown by the controller and doesn't participate in more barriers
|
||||
|
||||
case `aUser` ⇒
|
||||
|
|
@ -79,6 +79,7 @@ abstract class LeaderElectionSpec extends MultiNodeSpec(LeaderElectionMultiJvmSp
|
|||
// user marks the shutdown leader as DOWN
|
||||
cluster.down(leaderAddress)
|
||||
testConductor.enter("after-down", "completed")
|
||||
markNodeAsUnavailable(leaderAddress)
|
||||
|
||||
case _ if remainingRoles.contains(myself) ⇒
|
||||
// remaining cluster nodes, not shutdown
|
||||
|
|
@ -92,7 +93,6 @@ abstract class LeaderElectionSpec extends MultiNodeSpec(LeaderElectionMultiJvmSp
|
|||
testConductor.enter("completed")
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"be able to 're-elect' a single leader after leader has left" taggedAs LongRunningTest in {
|
||||
|
|
@ -103,5 +103,4 @@ abstract class LeaderElectionSpec extends MultiNodeSpec(LeaderElectionMultiJvmSp
|
|||
shutdownLeaderAndVerifyNewLeader(alreadyShutdown = 1)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,25 +18,25 @@ object MembershipChangeListenerExitingMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
leader-actions-frequency = 5000 ms # increase the leader action task frequency
|
||||
unreachable-nodes-reaper-frequency = 30000 ms # turn "off" reaping to unreachable node set
|
||||
leader-actions-interval = 5 s # increase the leader action task interval
|
||||
unreachable-nodes-reaper-interval = 30 s # turn "off" reaping to unreachable node set
|
||||
}
|
||||
""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class MembershipChangeListenerExitingMultiJvmNode1 extends MembershipChangeListenerExitingSpec
|
||||
class MembershipChangeListenerExitingMultiJvmNode2 extends MembershipChangeListenerExitingSpec
|
||||
class MembershipChangeListenerExitingMultiJvmNode3 extends MembershipChangeListenerExitingSpec
|
||||
class MembershipChangeListenerExitingMultiJvmNode1 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerExitingMultiJvmNode2 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerExitingMultiJvmNode3 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class MembershipChangeListenerExitingSpec
|
||||
extends MultiNodeSpec(MembershipChangeListenerExitingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class MembershipChangeListenerExitingSpec extends MultiNodeSpec(MembershipChangeListenerExitingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import MembershipChangeListenerExitingMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
|
@ -44,32 +44,29 @@ abstract class MembershipChangeListenerExitingSpec extends MultiNodeSpec(Members
|
|||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is EXITING" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(second, third) {
|
||||
cluster.join(firstAddress)
|
||||
runOn(first) {
|
||||
testConductor.enter("registered-listener")
|
||||
cluster.leave(secondAddress)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("registered-listener")
|
||||
}
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
testConductor.enter("rest-started")
|
||||
|
||||
runOn(third) {
|
||||
val exitingLatch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 3 && members.exists(_.status == MemberStatus.Exiting))
|
||||
if (members.size == 3 && members.exists(m ⇒ m.address == secondAddress && m.status == MemberStatus.Exiting))
|
||||
exitingLatch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("registered-listener")
|
||||
exitingLatch.await
|
||||
}
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,85 +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 org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
||||
object MembershipChangeListenerJoinAndUpMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
gossip-frequency = 1000 ms
|
||||
leader-actions-frequency = 5000 ms # increase the leader action task frequency
|
||||
}
|
||||
""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class MembershipChangeListenerJoinAndUpMultiJvmNode1 extends MembershipChangeListenerJoinAndUpSpec
|
||||
class MembershipChangeListenerJoinAndUpMultiJvmNode2 extends MembershipChangeListenerJoinAndUpSpec
|
||||
|
||||
abstract class MembershipChangeListenerJoinAndUpSpec
|
||||
extends MultiNodeSpec(MembershipChangeListenerJoinAndUpMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender
|
||||
with BeforeAndAfter {
|
||||
|
||||
import MembershipChangeListenerJoinAndUpMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 2
|
||||
|
||||
after {
|
||||
testConductor.enter("after")
|
||||
}
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is JOINING and node is marked as UP by the leader" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
|
||||
runOn(first) {
|
||||
// JOINING
|
||||
val joinLatch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 2 && members.exists(_.status == MemberStatus.Joining)) // second node is not part of node ring anymore
|
||||
joinLatch.countDown()
|
||||
}
|
||||
})
|
||||
joinLatch.await
|
||||
cluster.convergence.isDefined must be(true)
|
||||
|
||||
// UP
|
||||
val upLatch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 2 && members.forall(_.status == MemberStatus.Up))
|
||||
upLatch.countDown()
|
||||
}
|
||||
})
|
||||
upLatch.await
|
||||
awaitCond(cluster.convergence.isDefined)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,61 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
||||
object MembershipChangeListenerJoinMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval to allow time checking for JOIN before leader moves it to UP
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class MembershipChangeListenerJoinMultiJvmNode1 extends MembershipChangeListenerJoinSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerJoinMultiJvmNode2 extends MembershipChangeListenerJoinSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class MembershipChangeListenerJoinSpec
|
||||
extends MultiNodeSpec(MembershipChangeListenerJoinMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import MembershipChangeListenerJoinMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is JOINING" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
val joinLatch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 2 && members.exists(_.status == MemberStatus.Joining)) // second node is not part of node ring anymore
|
||||
joinLatch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("registered-listener")
|
||||
|
||||
joinLatch.await
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("registered-listener")
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
|
||||
testConductor.enter("after")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -17,23 +17,23 @@ object MembershipChangeListenerLeavingMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster.leader-actions-frequency = 5000 ms
|
||||
akka.cluster.unreachable-nodes-reaper-frequency = 30000 ms # turn "off" reaping to unreachable node set
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster.leader-actions-interval = 5 s
|
||||
akka.cluster.unreachable-nodes-reaper-interval = 30 s
|
||||
"""))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class MembershipChangeListenerLeavingMultiJvmNode1 extends MembershipChangeListenerLeavingSpec
|
||||
class MembershipChangeListenerLeavingMultiJvmNode2 extends MembershipChangeListenerLeavingSpec
|
||||
class MembershipChangeListenerLeavingMultiJvmNode3 extends MembershipChangeListenerLeavingSpec
|
||||
class MembershipChangeListenerLeavingMultiJvmNode1 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerLeavingMultiJvmNode2 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerLeavingMultiJvmNode3 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class MembershipChangeListenerLeavingSpec
|
||||
extends MultiNodeSpec(MembershipChangeListenerLeavingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class MembershipChangeListenerLeavingSpec extends MultiNodeSpec(MembershipChangeListenerLeavingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import MembershipChangeListenerLeavingMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
|
@ -41,32 +41,29 @@ abstract class MembershipChangeListenerLeavingSpec extends MultiNodeSpec(Members
|
|||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is LEAVING" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(second, third) {
|
||||
cluster.join(firstAddress)
|
||||
runOn(first) {
|
||||
testConductor.enter("registered-listener")
|
||||
cluster.leave(secondAddress)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("registered-listener")
|
||||
}
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
testConductor.enter("rest-started")
|
||||
|
||||
runOn(third) {
|
||||
val latch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 3 && members.exists(_.status == MemberStatus.Leaving))
|
||||
if (members.size == 3 && members.exists(m ⇒ m.address == secondAddress && m.status == MemberStatus.Leaving))
|
||||
latch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("registered-listener")
|
||||
latch.await
|
||||
}
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,79 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
|
||||
object MembershipChangeListenerUpMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class MembershipChangeListenerUpMultiJvmNode1 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerUpMultiJvmNode2 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy
|
||||
class MembershipChangeListenerUpMultiJvmNode3 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class MembershipChangeListenerUpSpec
|
||||
extends MultiNodeSpec(MembershipChangeListenerUpMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import MembershipChangeListenerUpMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A set of connected cluster systems" must {
|
||||
|
||||
"(when two nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first)
|
||||
|
||||
runOn(first, second) {
|
||||
val latch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 2 && members.forall(_.status == MemberStatus.Up))
|
||||
latch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("listener-1-registered")
|
||||
cluster.join(firstAddress)
|
||||
latch.await
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
testConductor.enter("listener-1-registered")
|
||||
}
|
||||
|
||||
testConductor.enter("after-1")
|
||||
}
|
||||
|
||||
"(when three nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
|
||||
|
||||
val latch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 3 && members.forall(_.status == MemberStatus.Up))
|
||||
latch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("listener-2-registered")
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
|
||||
latch.await
|
||||
|
||||
testConductor.enter("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -5,7 +5,7 @@ package akka.cluster
|
|||
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.Address
|
||||
import akka.actor.{Address, ExtendedActorSystem}
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
|
|
@ -15,11 +15,12 @@ import akka.util.Duration
|
|||
object MultiNodeClusterSpec {
|
||||
def clusterConfig: Config = ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
auto-down = off
|
||||
gossip-frequency = 200 ms
|
||||
leader-actions-frequency = 200 ms
|
||||
unreachable-nodes-reaper-frequency = 200 ms
|
||||
periodic-tasks-initial-delay = 300 ms
|
||||
auto-down = off
|
||||
gossip-interval = 200 ms
|
||||
leader-actions-interval = 200 ms
|
||||
unreachable-nodes-reaper-interval = 200 ms
|
||||
periodic-tasks-initial-delay = 300 ms
|
||||
nr-of-deputy-nodes = 2
|
||||
}
|
||||
akka.test {
|
||||
single-expect-default = 5 s
|
||||
|
|
@ -27,9 +28,57 @@ object MultiNodeClusterSpec {
|
|||
""")
|
||||
}
|
||||
|
||||
trait MultiNodeClusterSpec { self: MultiNodeSpec ⇒
|
||||
trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec ⇒
|
||||
|
||||
def cluster: Cluster = Cluster(system)
|
||||
override def initialParticipants = roles.size
|
||||
|
||||
/**
|
||||
* The cluster node instance. Needs to be lazily created.
|
||||
*/
|
||||
private lazy val clusterNode = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector)
|
||||
|
||||
/**
|
||||
* Get the cluster node to use.
|
||||
*/
|
||||
def cluster: Cluster = clusterNode
|
||||
|
||||
/**
|
||||
* Use this method instead of 'cluster.self'
|
||||
* for the initial startup of the cluster node.
|
||||
*/
|
||||
def startClusterNode(): Unit = cluster.self
|
||||
|
||||
/**
|
||||
* Initialize the cluster with the specified member
|
||||
* nodes (roles). First node will be started first
|
||||
* and others will join the first.
|
||||
*/
|
||||
def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.toSeq)
|
||||
|
||||
/**
|
||||
* Initialize the cluster of the specified member
|
||||
* nodes (roles) and wait until all joined and `Up`.
|
||||
* First node will be started first and others will join
|
||||
* the first.
|
||||
*/
|
||||
def awaitClusterUp(roles: RoleName*): Unit = {
|
||||
awaitStartCluster(true, roles.toSeq)
|
||||
}
|
||||
|
||||
private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = {
|
||||
runOn(roles.head) {
|
||||
// make sure that the node-to-join is started before other join
|
||||
startClusterNode()
|
||||
}
|
||||
testConductor.enter(roles.head.name + "-started")
|
||||
if (roles.tail.contains(myself)) {
|
||||
cluster.join(node(roles.head).address)
|
||||
}
|
||||
if (upConvergence && roles.contains(myself)) {
|
||||
awaitUpConvergence(numberOfMembers = roles.length)
|
||||
}
|
||||
testConductor.enter(roles.map(_.name).mkString("-") + "-joined")
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert that the member addresses match the expected addresses in the
|
||||
|
|
@ -76,6 +125,17 @@ trait MultiNodeClusterSpec { self: MultiNodeSpec ⇒
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait until the specified nodes have seen the same gossip overview.
|
||||
*/
|
||||
def awaitSeenSameState(addresses: Seq[Address]): Unit = {
|
||||
awaitCond {
|
||||
val seen = cluster.latestGossip.overview.seen
|
||||
val seenVectorClocks = addresses.flatMap(seen.get(_))
|
||||
seenVectorClocks.size == addresses.size && seenVectorClocks.toSet.size == 1
|
||||
}
|
||||
}
|
||||
|
||||
def roleOfLeader(nodesInCluster: Seq[RoleName]): RoleName = {
|
||||
nodesInCluster.length must not be (0)
|
||||
nodesInCluster.sorted.head
|
||||
|
|
|
|||
|
|
@ -1,76 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
||||
object NodeJoinAndUpMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
gossip-frequency = 1000 ms
|
||||
leader-actions-frequency = 5000 ms # increase the leader action task frequency
|
||||
}
|
||||
""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class NodeJoinAndUpMultiJvmNode1 extends NodeJoinAndUpSpec
|
||||
class NodeJoinAndUpMultiJvmNode2 extends NodeJoinAndUpSpec
|
||||
|
||||
abstract class NodeJoinAndUpSpec
|
||||
extends MultiNodeSpec(NodeJoinAndUpMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender
|
||||
with BeforeAndAfter {
|
||||
|
||||
import NodeJoinAndUpMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 2
|
||||
|
||||
after {
|
||||
testConductor.enter("after")
|
||||
}
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A first cluster node with a 'node-to-join' config set to empty string (singleton cluster)" must {
|
||||
|
||||
"be a singleton cluster when started up" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
awaitCond(cluster.isSingletonCluster)
|
||||
awaitUpConvergence(numberOfMembers = 1)
|
||||
cluster.isLeader must be(true)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"A second cluster node" must {
|
||||
"join the cluster as JOINING - when sending a 'Join' command - and then be moved to UP by the leader" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
|
||||
awaitCond(cluster.latestGossip.members.exists { member ⇒ member.address == secondAddress && member.status == MemberStatus.Joining })
|
||||
|
||||
awaitCond(
|
||||
cluster.latestGossip.members.exists { member ⇒ member.address == secondAddress && member.status == MemberStatus.Up },
|
||||
30.seconds.dilated) // waiting for the leader to move from JOINING -> UP (frequency set to 5 sec in config)
|
||||
|
||||
cluster.latestGossip.members.size must be(2)
|
||||
awaitCond(cluster.convergence.isDefined)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,51 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
||||
object NodeJoinMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class NodeJoinMultiJvmNode1 extends NodeJoinSpec with FailureDetectorPuppetStrategy
|
||||
class NodeJoinMultiJvmNode2 extends NodeJoinSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class NodeJoinSpec
|
||||
extends MultiNodeSpec(NodeJoinMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import NodeJoinMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A cluster node" must {
|
||||
"join another cluster and get status JOINING - when sending a 'Join' command" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
startClusterNode()
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
|
||||
awaitCond(cluster.latestGossip.members.exists { member ⇒ member.address == secondAddress && member.status == MemberStatus.Joining })
|
||||
|
||||
testConductor.enter("after")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
|
|
@ -19,16 +18,16 @@ object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig
|
|||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class NodeLeavingAndExitingAndBeingRemovedSpec
|
||||
extends MultiNodeSpec(NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class NodeLeavingAndExitingAndBeingRemovedSpec extends MultiNodeSpec(NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec)
|
||||
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
|
@ -37,18 +36,10 @@ abstract class NodeLeavingAndExitingAndBeingRemovedSpec extends MultiNodeSpec(No
|
|||
|
||||
"A node that is LEAVING a non-singleton cluster" must {
|
||||
|
||||
"be moved to EXITING and then to REMOVED by the reaper" taggedAs LongRunningTest in {
|
||||
// FIXME make it work and remove ignore
|
||||
"be moved to EXITING and then to REMOVED by the reaper" taggedAs LongRunningTest ignore {
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
runOn(second, third) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
testConductor.enter("rest-started")
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
|
|
@ -18,43 +17,35 @@ object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
leader-actions-frequency = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state
|
||||
unreachable-nodes-reaper-frequency = 30 s # turn "off" reaping to unreachable node set
|
||||
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state
|
||||
unreachable-nodes-reaper-interval = 30 s
|
||||
}
|
||||
""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class NodeLeavingAndExitingMultiJvmNode1 extends NodeLeavingAndExitingSpec
|
||||
class NodeLeavingAndExitingMultiJvmNode2 extends NodeLeavingAndExitingSpec
|
||||
class NodeLeavingAndExitingMultiJvmNode3 extends NodeLeavingAndExitingSpec
|
||||
class NodeLeavingAndExitingMultiJvmNode1 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy
|
||||
class NodeLeavingAndExitingMultiJvmNode2 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy
|
||||
class NodeLeavingAndExitingMultiJvmNode3 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class NodeLeavingAndExitingSpec
|
||||
extends MultiNodeSpec(NodeLeavingAndExitingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class NodeLeavingAndExitingSpec extends MultiNodeSpec(NodeLeavingAndExitingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import NodeLeavingAndExitingMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A node that is LEAVING a non-singleton cluster" must {
|
||||
|
||||
"be moved to EXITING by the leader" taggedAs LongRunningTest in {
|
||||
// FIXME make it work and remove ignore
|
||||
"be moved to EXITING by the leader" taggedAs LongRunningTest ignore {
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
runOn(second, third) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
testConductor.enter("rest-started")
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
|
|
@ -64,7 +55,7 @@ abstract class NodeLeavingAndExitingSpec extends MultiNodeSpec(NodeLeavingAndExi
|
|||
runOn(first, third) {
|
||||
|
||||
// 1. Verify that 'second' node is set to LEAVING
|
||||
// We have set the 'leader-actions-frequency' to 5 seconds to make sure that we get a
|
||||
// We have set the 'leader-actions-interval' to 5 seconds to make sure that we get a
|
||||
// chance to test the LEAVING state before the leader moves the node to EXITING
|
||||
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving)) // wait on LEAVING
|
||||
val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving) // verify node that left
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
|
|
@ -17,41 +16,30 @@ object NodeLeavingMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster.leader-actions-frequency = 5 s
|
||||
akka.cluster.unreachable-nodes-reaper-frequency = 30 s # turn "off" reaping to unreachable node set
|
||||
"""))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
.withFallback(ConfigFactory.parseString("akka.cluster.unreachable-nodes-reaper-frequency = 30 s"))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class NodeLeavingMultiJvmNode1 extends NodeLeavingSpec
|
||||
class NodeLeavingMultiJvmNode2 extends NodeLeavingSpec
|
||||
class NodeLeavingMultiJvmNode3 extends NodeLeavingSpec
|
||||
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 {
|
||||
|
||||
abstract class NodeLeavingSpec extends MultiNodeSpec(NodeLeavingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import NodeLeavingMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A node that is LEAVING a non-singleton cluster" must {
|
||||
|
||||
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest in {
|
||||
// FIXME make it work and remove ignore
|
||||
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest ignore {
|
||||
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
runOn(second, third) {
|
||||
cluster.join(firstAddress)
|
||||
}
|
||||
awaitUpConvergence(numberOfMembers = 3)
|
||||
testConductor.enter("rest-started")
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
|
|
@ -15,22 +14,18 @@ object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
|
|||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class NodeMembershipMultiJvmNode1 extends NodeMembershipSpec
|
||||
class NodeMembershipMultiJvmNode2 extends NodeMembershipSpec
|
||||
class NodeMembershipMultiJvmNode3 extends NodeMembershipSpec
|
||||
class NodeMembershipMultiJvmNode1 extends NodeMembershipSpec with FailureDetectorPuppetStrategy
|
||||
class NodeMembershipMultiJvmNode2 extends NodeMembershipSpec with FailureDetectorPuppetStrategy
|
||||
class NodeMembershipMultiJvmNode3 extends NodeMembershipSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class NodeMembershipSpec
|
||||
extends MultiNodeSpec(NodeMembershipMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
abstract class NodeMembershipSpec extends MultiNodeSpec(NodeMembershipMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import NodeMembershipMultiJvmSpec._
|
||||
|
||||
override def initialParticipants = 3
|
||||
|
||||
after {
|
||||
testConductor.enter("after")
|
||||
}
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
|
@ -41,7 +36,7 @@ abstract class NodeMembershipSpec extends MultiNodeSpec(NodeMembershipMultiJvmSp
|
|||
|
||||
// make sure that the node-to-join is started before other join
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
startClusterNode()
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
|
|
@ -55,6 +50,7 @@ abstract class NodeMembershipSpec extends MultiNodeSpec(NodeMembershipMultiJvmSp
|
|||
awaitCond(cluster.convergence.isDefined)
|
||||
}
|
||||
|
||||
testConductor.enter("after-1")
|
||||
}
|
||||
|
||||
"(when three nodes) start gossiping to each other so that all nodes gets the same gossip info" taggedAs LongRunningTest in {
|
||||
|
|
@ -70,7 +66,7 @@ abstract class NodeMembershipSpec extends MultiNodeSpec(NodeMembershipMultiJvmSp
|
|||
}
|
||||
awaitCond(cluster.convergence.isDefined)
|
||||
|
||||
testConductor.enter("after-2")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
import scala.collection.immutable.SortedSet
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
object NodeUpMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class NodeUpMultiJvmNode1 extends NodeUpSpec with FailureDetectorPuppetStrategy
|
||||
class NodeUpMultiJvmNode2 extends NodeUpSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class NodeUpSpec
|
||||
extends MultiNodeSpec(NodeUpMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import NodeUpMultiJvmSpec._
|
||||
|
||||
"A cluster node that is joining another cluster" must {
|
||||
"be moved to UP by the leader after a convergence" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second)
|
||||
|
||||
testConductor.enter("after-1")
|
||||
}
|
||||
|
||||
"be unaffected when joining again" taggedAs LongRunningTest in {
|
||||
|
||||
val unexpected = new AtomicReference[SortedSet[Member]]
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size != 2 || members.exists(_.status != MemberStatus.Up))
|
||||
unexpected.set(members)
|
||||
}
|
||||
})
|
||||
testConductor.enter("listener-registered")
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(node(first).address)
|
||||
}
|
||||
testConductor.enter("joined-again")
|
||||
|
||||
// let it run for a while to make sure that nothing bad happens
|
||||
for (n ← 1 to 20) {
|
||||
100.millis.dilated.sleep()
|
||||
unexpected.get must be(null)
|
||||
cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true)
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -4,20 +4,19 @@
|
|||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
||||
object NodeShutdownMultiJvmSpec extends MultiNodeConfig {
|
||||
object SingletonClusterMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
auto-down = on
|
||||
auto-down = on
|
||||
failure-detector.threshold = 4
|
||||
}
|
||||
""")).
|
||||
|
|
@ -25,46 +24,41 @@ object NodeShutdownMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
}
|
||||
|
||||
class NodeShutdownMultiJvmNode1 extends NodeShutdownSpec
|
||||
class NodeShutdownMultiJvmNode2 extends NodeShutdownSpec
|
||||
class SingletonClusterWithFailureDetectorPuppetMultiJvmNode1 extends SingletonClusterSpec with FailureDetectorPuppetStrategy
|
||||
class SingletonClusterWithFailureDetectorPuppetMultiJvmNode2 extends SingletonClusterSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class NodeShutdownSpec extends MultiNodeSpec(NodeShutdownMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender with BeforeAndAfter {
|
||||
import NodeShutdownMultiJvmSpec._
|
||||
class SingletonClusterWithAccrualFailureDetectorMultiJvmNode1 extends SingletonClusterSpec with AccrualFailureDetectorStrategy
|
||||
class SingletonClusterWithAccrualFailureDetectorMultiJvmNode2 extends SingletonClusterSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
override def initialParticipants = 2
|
||||
abstract class SingletonClusterSpec
|
||||
extends MultiNodeSpec(SingletonClusterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
after {
|
||||
testConductor.enter("after")
|
||||
}
|
||||
import SingletonClusterMultiJvmSpec._
|
||||
|
||||
"A cluster of 2 nodes" must {
|
||||
|
||||
"not be singleton cluster when joined" taggedAs LongRunningTest in {
|
||||
// make sure that the node-to-join is started before other join
|
||||
runOn(first) {
|
||||
cluster.self
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(node(first).address)
|
||||
}
|
||||
awaitUpConvergence(numberOfMembers = 2)
|
||||
awaitClusterUp(first, second)
|
||||
cluster.isSingletonCluster must be(false)
|
||||
assertLeader(first, second)
|
||||
|
||||
testConductor.enter("after-1")
|
||||
}
|
||||
|
||||
"become singleton cluster when one node is shutdown" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val secondAddress = node(second).address
|
||||
testConductor.shutdown(second, 0)
|
||||
testConductor.removeNode(second)
|
||||
|
||||
markNodeAsUnavailable(secondAddress)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
|
||||
cluster.isSingletonCluster must be(true)
|
||||
assertLeader(first)
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import scala.collection.immutable.SortedSet
|
||||
|
||||
object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
val fifth = role("fifth")
|
||||
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
gossip-interval = 400 ms
|
||||
nr-of-deputy-nodes = 0
|
||||
}
|
||||
akka.loglevel = DEBUG
|
||||
"""))
|
||||
}
|
||||
|
||||
class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class SunnyWeatherSpec
|
||||
extends MultiNodeSpec(SunnyWeatherMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import SunnyWeatherMultiJvmSpec._
|
||||
|
||||
"A normal cluster" must {
|
||||
"be healthy" taggedAs LongRunningTest in {
|
||||
|
||||
// start some
|
||||
awaitClusterUp(first, second, third)
|
||||
runOn(first, second, third) {
|
||||
log.info("3 joined")
|
||||
}
|
||||
|
||||
// add a few more
|
||||
awaitClusterUp(roles: _*)
|
||||
log.info("5 joined")
|
||||
|
||||
val unexpected = new AtomicReference[SortedSet[Member]]
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
// we don't expected any changes to the cluster
|
||||
unexpected.set(members)
|
||||
}
|
||||
})
|
||||
|
||||
for (n ← 1 to 30) {
|
||||
testConductor.enter("period-" + n)
|
||||
unexpected.get must be(null)
|
||||
awaitUpConvergence(roles.size)
|
||||
assertLeaderIn(roles)
|
||||
if (n % 5 == 0) log.info("Passed period [{}]", n)
|
||||
1.seconds.sleep
|
||||
}
|
||||
|
||||
testConductor.enter("after")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -17,7 +17,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
val conn = Address("akka", "", "localhost", 2552)
|
||||
val conn2 = Address("akka", "", "localhost", 2553)
|
||||
|
||||
def fakeTimeGenerator(timeIntervals: List[Long]): () ⇒ Long = {
|
||||
def fakeTimeGenerator(timeIntervals: Seq[Long]): () ⇒ Long = {
|
||||
var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) ⇒ acc ::: List[Long](acc.last + c))
|
||||
def timeGenerator(): Long = {
|
||||
val currentTime = times.head
|
||||
|
|
@ -27,22 +27,47 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
timeGenerator
|
||||
}
|
||||
|
||||
"return phi value of 0.0D on startup for each address" in {
|
||||
val fd = new AccrualFailureDetector(system, conn)
|
||||
fd.phi(conn) must be(0.0D)
|
||||
fd.phi(conn2) must be(0.0D)
|
||||
"return phi value of 0.0 on startup for each address, when no heartbeats" in {
|
||||
val fd = new AccrualFailureDetector(system)
|
||||
fd.phi(conn) must be(0.0)
|
||||
fd.phi(conn2) must be(0.0)
|
||||
}
|
||||
|
||||
"return phi based on guess when only one heartbeat" in {
|
||||
// 1 second ticks
|
||||
val timeInterval = Vector.fill(30)(1000L)
|
||||
val fd = new AccrualFailureDetector(system,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn)
|
||||
fd.phi(conn) must be > (0.0)
|
||||
// let time go
|
||||
for (n ← 2 to 8)
|
||||
fd.phi(conn) must be < (4.0)
|
||||
for (n ← 9 to 18)
|
||||
fd.phi(conn) must be < (8.0)
|
||||
|
||||
fd.phi(conn) must be > (8.0)
|
||||
}
|
||||
|
||||
"return phi value using first interval after second heartbeat" in {
|
||||
val timeInterval = List[Long](0, 100, 100, 100)
|
||||
val fd = new AccrualFailureDetector(system,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn)
|
||||
fd.phi(conn) must be > (0.0)
|
||||
fd.heartbeat(conn)
|
||||
fd.phi(conn) must be > (0.0)
|
||||
}
|
||||
|
||||
"mark node as available after a series of successful heartbeats" in {
|
||||
val timeInterval = List[Long](0, 1000, 100, 100)
|
||||
val ft = fakeTimeGenerator(timeInterval)
|
||||
|
||||
val fd = new AccrualFailureDetector(system, conn, timeMachine = ft)
|
||||
val fd = new AccrualFailureDetector(system,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn)
|
||||
|
||||
fd.heartbeat(conn)
|
||||
|
||||
fd.heartbeat(conn)
|
||||
|
||||
fd.isAvailable(conn) must be(true)
|
||||
|
|
@ -50,18 +75,13 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
|
||||
"mark node as dead after explicit removal of connection" in {
|
||||
val timeInterval = List[Long](0, 1000, 100, 100, 100)
|
||||
val ft = fakeTimeGenerator(timeInterval)
|
||||
|
||||
val fd = new AccrualFailureDetector(system, conn, timeMachine = ft)
|
||||
val fd = new AccrualFailureDetector(system,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn)
|
||||
|
||||
fd.heartbeat(conn)
|
||||
|
||||
fd.heartbeat(conn)
|
||||
|
||||
fd.isAvailable(conn) must be(true)
|
||||
|
||||
fd.remove(conn)
|
||||
|
||||
fd.isAvailable(conn) must be(false)
|
||||
|
|
@ -69,14 +89,12 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
|
||||
"mark node as available after explicit removal of connection and receiving heartbeat again" in {
|
||||
val timeInterval = List[Long](0, 1000, 100, 1100, 1100, 1100, 1100, 1100, 100)
|
||||
val ft = fakeTimeGenerator(timeInterval)
|
||||
|
||||
val fd = new AccrualFailureDetector(system, conn, timeMachine = ft)
|
||||
val fd = new AccrualFailureDetector(system,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn) //0
|
||||
|
||||
fd.heartbeat(conn) //1000
|
||||
|
||||
fd.heartbeat(conn) //1100
|
||||
|
||||
fd.isAvailable(conn) must be(true) //2200
|
||||
|
|
@ -87,9 +105,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
|
||||
// it receives heartbeat from an explicitly removed node
|
||||
fd.heartbeat(conn) //4400
|
||||
|
||||
fd.heartbeat(conn) //5500
|
||||
|
||||
fd.heartbeat(conn) //6600
|
||||
|
||||
fd.isAvailable(conn) must be(true) //6700
|
||||
|
|
@ -98,40 +114,29 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
"mark node as dead if heartbeat are missed" in {
|
||||
val timeInterval = List[Long](0, 1000, 100, 100, 5000)
|
||||
val ft = fakeTimeGenerator(timeInterval)
|
||||
|
||||
val fd = new AccrualFailureDetector(system, conn, threshold = 3, timeMachine = ft)
|
||||
val fd = new AccrualFailureDetector(system, threshold = 3,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn) //0
|
||||
|
||||
fd.heartbeat(conn) //1000
|
||||
|
||||
fd.heartbeat(conn) //1100
|
||||
|
||||
fd.isAvailable(conn) must be(true) //1200
|
||||
|
||||
fd.isAvailable(conn) must be(false) //6200
|
||||
}
|
||||
|
||||
"mark node as available if it starts heartbeat again after being marked dead due to detection of failure" in {
|
||||
val timeInterval = List[Long](0, 1000, 100, 1100, 5000, 100, 1000, 100, 100)
|
||||
val ft = fakeTimeGenerator(timeInterval)
|
||||
|
||||
val fd = new AccrualFailureDetector(system, conn, threshold = 3, timeMachine = ft)
|
||||
val fd = new AccrualFailureDetector(system, threshold = 3,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
fd.heartbeat(conn) //0
|
||||
|
||||
fd.heartbeat(conn) //1000
|
||||
|
||||
fd.heartbeat(conn) //1100
|
||||
|
||||
fd.isAvailable(conn) must be(true) //1200
|
||||
|
||||
fd.isAvailable(conn) must be(false) //6200
|
||||
|
||||
fd.heartbeat(conn) //6300
|
||||
|
||||
fd.heartbeat(conn) //7300
|
||||
|
||||
fd.heartbeat(conn) //7400
|
||||
|
||||
fd.isAvailable(conn) must be(true) //7500
|
||||
|
|
@ -139,8 +144,8 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
|
||||
"use maxSampleSize heartbeats" in {
|
||||
val timeInterval = List[Long](0, 100, 100, 100, 100, 600, 1000, 1000, 1000, 1000, 1000)
|
||||
val ft = fakeTimeGenerator(timeInterval)
|
||||
val fd = new AccrualFailureDetector(system, conn, maxSampleSize = 3, timeMachine = ft)
|
||||
val fd = new AccrualFailureDetector(system, maxSampleSize = 3,
|
||||
timeMachine = fakeTimeGenerator(timeInterval))
|
||||
|
||||
// 100 ms interval
|
||||
fd.heartbeat(conn) //0
|
||||
|
|
@ -156,5 +161,6 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
val phi2 = fd.phi(conn) //5000
|
||||
phi2 must be(phi1.plusOrMinus(0.001))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,11 +18,12 @@ class ClusterConfigSpec extends AkkaSpec {
|
|||
import settings._
|
||||
FailureDetectorThreshold must be(8)
|
||||
FailureDetectorMaxSampleSize must be(1000)
|
||||
FailureDetectorImplementationClass must be(None)
|
||||
NodeToJoin must be(None)
|
||||
PeriodicTasksInitialDelay must be(1 seconds)
|
||||
GossipFrequency must be(1 second)
|
||||
LeaderActionsFrequency must be(1 second)
|
||||
UnreachableNodesReaperFrequency must be(1 second)
|
||||
GossipInterval must be(1 second)
|
||||
LeaderActionsInterval must be(1 second)
|
||||
UnreachableNodesReaperInterval must be(1 second)
|
||||
NrOfGossipDaemons must be(4)
|
||||
NrOfDeputyNodes must be(3)
|
||||
AutoDown must be(true)
|
||||
|
|
|
|||
230
akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
Normal file
230
akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
Normal file
|
|
@ -0,0 +1,230 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.util.duration._
|
||||
import akka.util.Duration
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.actor.Address
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.scalatest.BeforeAndAfter
|
||||
|
||||
object ClusterSpec {
|
||||
val config = """
|
||||
akka.cluster {
|
||||
auto-down = off
|
||||
nr-of-deputy-nodes = 3
|
||||
periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks
|
||||
}
|
||||
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
akka.remote.netty.port = 0
|
||||
# akka.loglevel = DEBUG
|
||||
"""
|
||||
|
||||
case class GossipTo(address: Address)
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
||||
import ClusterSpec._
|
||||
|
||||
val deterministicRandom = new AtomicInteger
|
||||
|
||||
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], new FailureDetectorPuppet(system)) {
|
||||
|
||||
override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = {
|
||||
if (addresses.isEmpty) None
|
||||
else Some(addresses.toSeq(deterministicRandom.getAndIncrement % addresses.size))
|
||||
}
|
||||
|
||||
override def gossipTo(address: Address): Unit = {
|
||||
if (address == self.address) {
|
||||
super.gossipTo(address)
|
||||
}
|
||||
// represent the gossip with a message to be used in asserts
|
||||
testActor ! GossipTo(address)
|
||||
}
|
||||
|
||||
@volatile
|
||||
var _gossipToUnreachableProbablity = 0.0
|
||||
|
||||
override def gossipToUnreachableProbablity(membersSize: Int, unreachableSize: Int): Double = {
|
||||
if (_gossipToUnreachableProbablity < 0.0) super.gossipToUnreachableProbablity(membersSize, unreachableSize)
|
||||
else _gossipToUnreachableProbablity
|
||||
}
|
||||
|
||||
@volatile
|
||||
var _gossipToDeputyProbablity = 0.0
|
||||
|
||||
override def gossipToDeputyProbablity(membersSize: Int, unreachableSize: Int, deputySize: Int): Double = {
|
||||
if (_gossipToDeputyProbablity < 0.0) super.gossipToDeputyProbablity(membersSize, unreachableSize, deputySize)
|
||||
else _gossipToDeputyProbablity
|
||||
}
|
||||
|
||||
@volatile
|
||||
var _unavailable: Set[Address] = Set.empty
|
||||
|
||||
override val failureDetector = new FailureDetectorPuppet(system) {
|
||||
override def isAvailable(connection: Address): Boolean = {
|
||||
if (_unavailable.contains(connection)) false
|
||||
else super.isAvailable(connection)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
val selfAddress = cluster.self.address
|
||||
val addresses = IndexedSeq(
|
||||
selfAddress,
|
||||
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 1),
|
||||
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 2),
|
||||
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 3),
|
||||
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 4),
|
||||
Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 5))
|
||||
|
||||
def memberStatus(address: Address): Option[MemberStatus] =
|
||||
cluster.latestGossip.members.collectFirst { case m if m.address == address ⇒ m.status }
|
||||
|
||||
before {
|
||||
cluster._gossipToUnreachableProbablity = 0.0
|
||||
cluster._gossipToDeputyProbablity = 0.0
|
||||
cluster._unavailable = Set.empty
|
||||
deterministicRandom.set(0)
|
||||
}
|
||||
|
||||
"A Cluster" must {
|
||||
|
||||
"initially be singleton cluster and reach convergence after first gossip" in {
|
||||
cluster.isSingletonCluster must be(true)
|
||||
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress))
|
||||
memberStatus(selfAddress) must be(Some(MemberStatus.Joining))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
cluster.gossip()
|
||||
expectMsg(GossipTo(selfAddress))
|
||||
awaitCond(cluster.convergence.isDefined)
|
||||
memberStatus(selfAddress) must be(Some(MemberStatus.Joining))
|
||||
cluster.leaderActions()
|
||||
memberStatus(selfAddress) must be(Some(MemberStatus.Up))
|
||||
}
|
||||
|
||||
"accept a joining node" in {
|
||||
cluster.joining(addresses(1))
|
||||
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress, addresses(1)))
|
||||
memberStatus(addresses(1)) must be(Some(MemberStatus.Joining))
|
||||
// FIXME why is it still convergence immediately after joining?
|
||||
//cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
"accept a few more joining nodes" in {
|
||||
for (a ← addresses.drop(2)) {
|
||||
cluster.joining(a)
|
||||
memberStatus(a) must be(Some(MemberStatus.Joining))
|
||||
}
|
||||
cluster.latestGossip.members.map(_.address) must be(addresses.toSet)
|
||||
}
|
||||
|
||||
"order members by host and port" in {
|
||||
// note the importance of using toSeq before map, otherwise it will not preserve the order
|
||||
cluster.latestGossip.members.toSeq.map(_.address) must be(addresses.toSeq)
|
||||
}
|
||||
|
||||
"gossip to random live node" in {
|
||||
cluster.latestGossip.members
|
||||
cluster.gossip()
|
||||
cluster.gossip()
|
||||
cluster.gossip()
|
||||
cluster.gossip()
|
||||
|
||||
expectMsg(GossipTo(addresses(1)))
|
||||
expectMsg(GossipTo(addresses(2)))
|
||||
expectMsg(GossipTo(addresses(3)))
|
||||
expectMsg(GossipTo(addresses(4)))
|
||||
|
||||
expectNoMsg(1 second)
|
||||
}
|
||||
|
||||
"use certain probability for gossiping to unreachable node depending on the number of unreachable and live nodes" in {
|
||||
cluster._gossipToUnreachableProbablity = -1.0 // use real impl
|
||||
cluster.gossipToUnreachableProbablity(10, 1) must be < (cluster.gossipToUnreachableProbablity(9, 1))
|
||||
cluster.gossipToUnreachableProbablity(10, 1) must be < (cluster.gossipToUnreachableProbablity(10, 2))
|
||||
cluster.gossipToUnreachableProbablity(10, 5) must be < (cluster.gossipToUnreachableProbablity(10, 9))
|
||||
cluster.gossipToUnreachableProbablity(0, 10) must be <= (1.0)
|
||||
cluster.gossipToUnreachableProbablity(1, 10) must be <= (1.0)
|
||||
cluster.gossipToUnreachableProbablity(10, 0) must be(0.0 plusOrMinus (0.0001))
|
||||
cluster.gossipToUnreachableProbablity(0, 0) must be(0.0 plusOrMinus (0.0001))
|
||||
}
|
||||
|
||||
"use certain probability for gossiping to deputy node depending on the number of unreachable and live nodes" in {
|
||||
cluster._gossipToDeputyProbablity = -1.0 // use real impl
|
||||
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(9, 1, 2))
|
||||
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 2))
|
||||
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 3))
|
||||
cluster.gossipToDeputyProbablity(10, 5, 5) must be < (cluster.gossipToDeputyProbablity(10, 9, 5))
|
||||
cluster.gossipToDeputyProbablity(0, 10, 0) must be <= (1.0)
|
||||
cluster.gossipToDeputyProbablity(1, 10, 1) must be <= (1.0)
|
||||
cluster.gossipToDeputyProbablity(10, 0, 0) must be(0.0 plusOrMinus (0.0001))
|
||||
cluster.gossipToDeputyProbablity(0, 0, 0) must be(0.0 plusOrMinus (0.0001))
|
||||
cluster.gossipToDeputyProbablity(4, 0, 4) must be(1.0 plusOrMinus (0.0001))
|
||||
cluster.gossipToDeputyProbablity(3, 7, 4) must be(1.0 plusOrMinus (0.0001))
|
||||
}
|
||||
|
||||
"gossip to duputy node" in {
|
||||
cluster._gossipToDeputyProbablity = 1.0 // always
|
||||
|
||||
// we have configured 2 deputy nodes
|
||||
cluster.gossip() // 1 is deputy
|
||||
cluster.gossip() // 2 is deputy
|
||||
cluster.gossip() // 3 is deputy
|
||||
cluster.gossip() // 4 is not deputy, and therefore a deputy is also used
|
||||
|
||||
expectMsg(GossipTo(addresses(1)))
|
||||
expectMsg(GossipTo(addresses(2)))
|
||||
expectMsg(GossipTo(addresses(3)))
|
||||
expectMsg(GossipTo(addresses(4)))
|
||||
// and the extra gossip to deputy
|
||||
expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3)))
|
||||
|
||||
expectNoMsg(1 second)
|
||||
|
||||
}
|
||||
|
||||
"gossip to random unreachable node" in {
|
||||
val dead = Set(addresses(1))
|
||||
cluster._unavailable = dead
|
||||
cluster._gossipToUnreachableProbablity = 1.0 // always
|
||||
|
||||
cluster.reapUnreachableMembers()
|
||||
cluster.latestGossip.overview.unreachable.map(_.address) must be(dead)
|
||||
|
||||
cluster.gossip()
|
||||
|
||||
expectMsg(GossipTo(addresses(2))) // first available
|
||||
expectMsg(GossipTo(addresses(1))) // the unavailable
|
||||
|
||||
expectNoMsg(1 second)
|
||||
}
|
||||
|
||||
"gossip to random deputy node if number of live nodes is less than number of deputy nodes" in {
|
||||
cluster._gossipToDeputyProbablity = -1.0 // real impl
|
||||
// 0 and 2 still alive
|
||||
val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5))
|
||||
cluster._unavailable = dead
|
||||
|
||||
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)
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,60 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import akka.actor.{ Address, ActorSystem }
|
||||
import akka.event.{ Logging, LogSource }
|
||||
|
||||
/**
|
||||
* User controllable "puppet" failure detector.
|
||||
*/
|
||||
class FailureDetectorPuppet(system: ActorSystem, settings: ClusterSettings) extends FailureDetector {
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
def this(system: ActorSystem) = this(system, new ClusterSettings(system.settings.config, system.name))
|
||||
|
||||
trait Status
|
||||
object Up extends Status
|
||||
object Down extends Status
|
||||
|
||||
implicit private val logSource: LogSource[AnyRef] = new LogSource[AnyRef] {
|
||||
def genString(o: AnyRef): String = o.getClass.getName
|
||||
override def getClazz(o: AnyRef): Class[_] = o.getClass
|
||||
}
|
||||
|
||||
private val log = Logging(system, this)
|
||||
|
||||
private val connections = new ConcurrentHashMap[Address, Status]
|
||||
|
||||
def markNodeAsUnavailable(connection: Address): this.type = {
|
||||
connections.put(connection, Down)
|
||||
this
|
||||
}
|
||||
|
||||
def markNodeAsAvailable(connection: Address): this.type = {
|
||||
connections.put(connection, Up)
|
||||
this
|
||||
}
|
||||
|
||||
def isAvailable(connection: Address): Boolean = connections.get(connection) match {
|
||||
case null ⇒
|
||||
log.debug("Adding cluster node [{}]", connection)
|
||||
connections.put(connection, Up)
|
||||
true
|
||||
case Up ⇒
|
||||
log.debug("isAvailable: Cluster node IS NOT available [{}]", connection)
|
||||
true
|
||||
case Down ⇒
|
||||
log.debug("isAvailable: Cluster node IS available [{}]", connection)
|
||||
false
|
||||
}
|
||||
|
||||
def heartbeat(connection: Address): Unit = log.debug("Heart beat from cluster node[{}]", connection)
|
||||
|
||||
def remove(connection: Address): Unit = {
|
||||
log.debug("Removing cluster node [{}]", connection)
|
||||
connections.remove(connection)
|
||||
}
|
||||
}
|
||||
42
akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala
Normal file
42
akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala
Normal file
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import org.scalatest.WordSpec
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import akka.actor.Address
|
||||
import scala.collection.immutable.SortedSet
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class GossipSpec extends WordSpec with MustMatchers {
|
||||
|
||||
"A Gossip" must {
|
||||
|
||||
"merge members by status priority" in {
|
||||
import MemberStatus._
|
||||
val a1 = Member(Address("akka", "sys", "a", 2552), Up)
|
||||
val a2 = Member(Address("akka", "sys", "a", 2552), Joining)
|
||||
val b1 = Member(Address("akka", "sys", "b", 2552), Up)
|
||||
val b2 = Member(Address("akka", "sys", "b", 2552), Removed)
|
||||
val c1 = Member(Address("akka", "sys", "c", 2552), Leaving)
|
||||
val c2 = Member(Address("akka", "sys", "c", 2552), Up)
|
||||
val d1 = Member(Address("akka", "sys", "d", 2552), Leaving)
|
||||
val d2 = Member(Address("akka", "sys", "d", 2552), Removed)
|
||||
|
||||
val g1 = Gossip(members = SortedSet(a1, b1, c1, d1))
|
||||
val g2 = Gossip(members = SortedSet(a2, b2, c2, d2))
|
||||
|
||||
val merged1 = g1 merge g2
|
||||
merged1.members must be(SortedSet(a1, b2, c1, d2))
|
||||
merged1.members.toSeq.map(_.status) must be(Seq(Up, Removed, Leaving, Removed))
|
||||
|
||||
val merged2 = g2 merge g1
|
||||
merged2.members must be(SortedSet(a1, b2, c1, d2))
|
||||
merged2.members.toSeq.map(_.status) must be(Seq(Up, Removed, Leaving, Removed))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -8,6 +8,7 @@ import org.scalatest.WordSpec
|
|||
import org.scalatest.matchers.MustMatchers
|
||||
import akka.actor.Address
|
||||
import scala.util.Random
|
||||
import scala.collection.immutable.SortedSet
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class MemberSpec extends WordSpec with MustMatchers {
|
||||
|
|
@ -26,6 +27,19 @@ class MemberSpec extends WordSpec with MustMatchers {
|
|||
val expected = IndexedSeq(m1, m2, m3, m4, m5)
|
||||
val shuffled = Random.shuffle(expected)
|
||||
shuffled.sorted must be(expected)
|
||||
(SortedSet.empty[Member] ++ shuffled).toIndexedSeq must be(expected)
|
||||
}
|
||||
|
||||
"have stable equals and hashCode" in {
|
||||
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Joining)
|
||||
val m2 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
|
||||
val m3 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
|
||||
|
||||
m1 must be(m2)
|
||||
m1.hashCode must be(m2.hashCode)
|
||||
|
||||
m3 must not be (m2)
|
||||
m3 must not be (m1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -90,6 +90,42 @@ strong {color: #1d3c52; }
|
|||
box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
|
||||
}
|
||||
|
||||
.warning {
|
||||
background-image: none;
|
||||
background-color: #fdf5d9;
|
||||
filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);
|
||||
padding: 14px;
|
||||
border-color: #ffffc4;
|
||||
-webkit-box-shadow: none;
|
||||
-moz-box-shadow: none;
|
||||
box-shadow: none;
|
||||
margin-bottom: 18px;
|
||||
position: relative;
|
||||
padding: 7px 15px;
|
||||
color: #404040;
|
||||
background-repeat: repeat-x;
|
||||
background-image: -khtml-gradient(linear, left top, left bottom, from(#ffffc4), to(#ffff00));
|
||||
background-image: -moz-linear-gradient(top, #ffffc4, #ffff00);
|
||||
background-image: -ms-linear-gradient(top, #ffffc4, #ffff00);
|
||||
background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #ffffc4), color-stop(100%, #ffff00));
|
||||
background-image: -webkit-linear-gradient(top, #ffffc4, #ffff00);
|
||||
background-image: -o-linear-gradient(top, #ffffc4, #ffff00);
|
||||
background-image: linear-gradient(top, #ffffc4, #ffff00);
|
||||
filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffc4', endColorstr='#ffff00', GradientType=0);
|
||||
text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
|
||||
border-color: #dff69a #ffff00 #E4C652;
|
||||
border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);
|
||||
text-shadow: 0 1px 0 rgba(255, 255, 255, 0.5);
|
||||
border-width: 1px;
|
||||
border-style: solid;
|
||||
-webkit-border-radius: 4px;
|
||||
-moz-border-radius: 4px;
|
||||
border-radius: 4px;
|
||||
-webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
|
||||
-moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
|
||||
box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25);
|
||||
}
|
||||
|
||||
.admonition p.admonition-title {
|
||||
color: rgba(0, 0, 0, 0.6);
|
||||
text-shadow: 0 1px 0 rgba(255, 255, 255, .7);
|
||||
|
|
|
|||
Binary file not shown.
|
Before Width: | Height: | Size: 50 KiB After Width: | Height: | Size: 41 KiB |
|
|
@ -65,7 +65,7 @@ When using JarJar, OneJar, Assembly or any jar-bundler
|
|||
Akka's configuration approach relies heavily on the notion of every
|
||||
module/jar having its own reference.conf file, all of these will be
|
||||
discovered by the configuration and loaded. Unfortunately this also means
|
||||
that if you put merge multiple jars into the same jar, you need to merge all the
|
||||
that if you put/merge multiple jars into the same jar, you need to merge all the
|
||||
reference.confs as well. Otherwise all defaults will be lost and Akka will not function.
|
||||
|
||||
Custom application.conf
|
||||
|
|
|
|||
|
|
@ -182,7 +182,7 @@ public class FaultHandlingTestBase {
|
|||
final TestProbe probe = new TestProbe(system);
|
||||
probe.watch(child);
|
||||
child.tell(new IllegalArgumentException());
|
||||
probe.expectMsg(new Terminated(child));
|
||||
probe.expectMsg(new Terminated(child, true));
|
||||
//#stop
|
||||
|
||||
//#escalate-kill
|
||||
|
|
@ -190,7 +190,7 @@ public class FaultHandlingTestBase {
|
|||
probe.watch(child);
|
||||
assert Await.result(ask(child, "get", 5000), timeout).equals(0);
|
||||
child.tell(new Exception());
|
||||
probe.expectMsg(new Terminated(child));
|
||||
probe.expectMsg(new Terminated(child, true));
|
||||
//#escalate-kill
|
||||
|
||||
//#escalate-restart
|
||||
|
|
|
|||
|
|
@ -43,7 +43,7 @@ Step Description
|
|||
9, 10, 11 and tells the ``Counter`` that there is no ``Storage``.
|
||||
12 The ``CounterService`` schedules a ``Reconnect`` message to itself.
|
||||
13, 14 When it receives the ``Reconnect`` message it creates a new ``Storage`` ...
|
||||
15, 16 and tells the the ``Counter`` to use the new ``Storage``
|
||||
15, 16 and tells the ``Counter`` to use the new ``Storage``
|
||||
=========== ==================================================================================
|
||||
|
||||
Full Source Code of the Fault Tolerance Sample (Java)
|
||||
|
|
|
|||
|
|
@ -24,4 +24,5 @@ Java API
|
|||
extending-akka
|
||||
zeromq
|
||||
microkernel
|
||||
testing
|
||||
howto
|
||||
|
|
|
|||
|
|
@ -187,7 +187,7 @@ It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4
|
|||
<dependency>
|
||||
<groupId>ch.qos.logback</groupId>
|
||||
<artifactId>logback-classic</artifactId>
|
||||
<version>1.0.0</version>
|
||||
<version>1.0.4</version>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
|
||||
|
|
@ -211,7 +211,7 @@ the first case and ``LoggerFactory.getLogger(String s)`` in the second).
|
|||
|
||||
.. note::
|
||||
|
||||
Beware that the the actor system’s name is appended to a :class:`String` log
|
||||
Beware that the actor system’s name is appended to a :class:`String` log
|
||||
source if the LoggingAdapter was created giving an :class:`ActorSystem` to
|
||||
the factory. If this is not intended, give a :class:`LoggingBus` instead as
|
||||
shown below:
|
||||
|
|
|
|||
|
|
@ -92,6 +92,14 @@ As you can see from the example above the following pattern is used to find an `
|
|||
|
||||
akka://<actorsystemname>@<hostname>:<port>/<actor path>
|
||||
|
||||
.. note::
|
||||
|
||||
In order to ensure serializability of ``Props`` when passing constructor
|
||||
arguments to the actor being created, do not make the factory a non-static
|
||||
inner class: this will inherently capture a reference to its enclosing
|
||||
object, which in most cases is not serializable. It is best to make a static
|
||||
inner class which implements :class:`UntypedActorFactory`.
|
||||
|
||||
Programmatic Remote Deployment
|
||||
------------------------------
|
||||
|
||||
|
|
@ -272,10 +280,6 @@ which holds the transport used (RemoteTransport) and the outbound address that i
|
|||
To intercept when an outbound client is shut down you listen to ``RemoteClientShutdown``
|
||||
which holds the transport used (RemoteTransport) and the outbound address that it was connected to (Address).
|
||||
|
||||
To intercept when an outbound message cannot be sent, you listen to ``RemoteClientWriteFailed`` which holds
|
||||
the payload that was not written (AnyRef), the cause of the failed send (Throwable),
|
||||
the transport used (RemoteTransport) and the outbound address that was the destination (Address).
|
||||
|
||||
For general outbound-related errors, that do not classify as any of the others, you can listen to ``RemoteClientError``,
|
||||
which holds the cause (Throwable), the transport used (RemoteTransport) and the outbound address (Address).
|
||||
|
||||
|
|
|
|||
14
akka-docs/java/testing.rst
Normal file
14
akka-docs/java/testing.rst
Normal file
|
|
@ -0,0 +1,14 @@
|
|||
.. _akka-testkit-java:
|
||||
|
||||
##############################
|
||||
Testing Actor Systems (Java)
|
||||
##############################
|
||||
|
||||
Due to the conciseness of test DSLs available for Scala, it may be a good idea
|
||||
to write the test suite in that language even if the main project is written in
|
||||
Java. If that is not desirable, you can also use :class:`TestKit` and friends
|
||||
from Java, albeit with more verbose syntax Munish Gupta has `published a nice
|
||||
post <http://www.akkaessentials.in/2012/05/using-testkit-with-java.html>`_
|
||||
showing several patterns you may find useful, and for reference documentation
|
||||
please refer to :ref:`akka-testkit` until that section has been ported over to
|
||||
cover Java in full.
|
||||
|
|
@ -586,7 +586,7 @@ What happens to the Message
|
|||
---------------------------
|
||||
|
||||
If an exception is thrown while a message is being processed (so taken of his
|
||||
mailbox and handed over the the receive), then this message will be lost. It is
|
||||
mailbox and handed over to the receive), then this message will be lost. It is
|
||||
important to understand that it is not put back on the mailbox. So if you want
|
||||
to retry processing of a message, you need to deal with it yourself by catching
|
||||
the exception and retry your flow. Make sure that you put a bound on the number
|
||||
|
|
|
|||
|
|
@ -651,7 +651,7 @@ What happens to the Message
|
|||
---------------------------
|
||||
|
||||
If an exception is thrown while a message is being processed (so taken of his
|
||||
mailbox and handed over the the receive), then this message will be lost. It is
|
||||
mailbox and handed over to the receive), then this message will be lost. It is
|
||||
important to understand that it is not put back on the mailbox. So if you want
|
||||
to retry processing of a message, you need to deal with it yourself by catching
|
||||
the exception and retry your flow. Make sure that you put a bound on the number
|
||||
|
|
|
|||
|
|
@ -111,7 +111,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
|
|||
//#stop
|
||||
watch(child) // have testActor watch “child”
|
||||
child ! new IllegalArgumentException // break it
|
||||
expectMsg(Terminated(child))
|
||||
expectMsg(Terminated(child)(existenceConfirmed = true))
|
||||
child.isTerminated must be(true)
|
||||
//#stop
|
||||
}
|
||||
|
|
@ -125,7 +125,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
|
|||
expectMsg(0)
|
||||
|
||||
child2 ! new Exception("CRASH") // escalate failure
|
||||
expectMsg(Terminated(child2))
|
||||
expectMsg(Terminated(child2)(existenceConfirmed = true))
|
||||
//#escalate-kill
|
||||
//#escalate-restart
|
||||
val supervisor2 = system.actorOf(Props[Supervisor2], "supervisor2")
|
||||
|
|
|
|||
|
|
@ -14,6 +14,8 @@ import akka.dispatch.Futures
|
|||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.testkit.ImplicitSender
|
||||
import akka.util.NonFatal
|
||||
|
||||
object TestkitDocSpec {
|
||||
case object Say42
|
||||
case object Unknown
|
||||
|
|
@ -208,7 +210,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
val probe = TestProbe()
|
||||
val future = probe.ref ? "hello"
|
||||
probe.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher
|
||||
probe.sender ! "world"
|
||||
probe.reply("world")
|
||||
assert(future.isCompleted && future.value == Some(Right("world")))
|
||||
//#test-probe-reply
|
||||
}
|
||||
|
|
@ -252,4 +254,22 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
//#event-filter
|
||||
}
|
||||
|
||||
"demonstrate TestKitBase" in {
|
||||
//#test-kit-base
|
||||
import akka.testkit.TestKitBase
|
||||
|
||||
class MyTest extends TestKitBase {
|
||||
implicit lazy val system = ActorSystem()
|
||||
|
||||
//#put-your-test-code-here
|
||||
val probe = TestProbe()
|
||||
probe.send(testActor, "hello")
|
||||
try expectMsg("hello") catch { case NonFatal(e) ⇒ system.shutdown(); throw e }
|
||||
//#put-your-test-code-here
|
||||
|
||||
system.shutdown()
|
||||
}
|
||||
//#test-kit-base
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -45,7 +45,7 @@ Step Description
|
|||
9, 10, 11 and tells the ``Counter`` that there is no ``Storage``.
|
||||
12 The ``CounterService`` schedules a ``Reconnect`` message to itself.
|
||||
13, 14 When it receives the ``Reconnect`` message it creates a new ``Storage`` ...
|
||||
15, 16 and tells the the ``Counter`` to use the new ``Storage``
|
||||
15, 16 and tells the ``Counter`` to use the new ``Storage``
|
||||
=========== ==================================================================================
|
||||
|
||||
Full Source Code of the Fault Tolerance Sample (Scala)
|
||||
|
|
|
|||
|
|
@ -230,7 +230,7 @@ It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4
|
|||
|
||||
.. code-block:: scala
|
||||
|
||||
lazy val logback = "ch.qos.logback" % "logback-classic" % "1.0.0" % "runtime"
|
||||
lazy val logback = "ch.qos.logback" % "logback-classic" % "1.0.4" % "runtime"
|
||||
|
||||
|
||||
You need to enable the Slf4jEventHandler in the 'event-handlers' element in
|
||||
|
|
@ -253,7 +253,7 @@ the first case and ``LoggerFactory.getLogger(s: String)`` in the second).
|
|||
|
||||
.. note::
|
||||
|
||||
Beware that the the actor system’s name is appended to a :class:`String` log
|
||||
Beware that the actor system’s name is appended to a :class:`String` log
|
||||
source if the LoggingAdapter was created giving an :class:`ActorSystem` to
|
||||
the factory. If this is not intended, give a :class:`LoggingBus` instead as
|
||||
shown below:
|
||||
|
|
|
|||
|
|
@ -105,6 +105,14 @@ Once you have configured the properties above you would do the following in code
|
|||
``SampleActor`` has to be available to the runtimes using it, i.e. the classloader of the
|
||||
actor systems has to have a JAR containing the class.
|
||||
|
||||
.. note::
|
||||
|
||||
In order to ensure serializability of ``Props`` when passing constructor
|
||||
arguments to the actor being created, do not make the factory an inner class:
|
||||
this will inherently capture a reference to its enclosing object, which in
|
||||
most cases is not serializable. It is best to create a factory method in the
|
||||
companion object of the actor’s class.
|
||||
|
||||
Programmatic Remote Deployment
|
||||
------------------------------
|
||||
|
||||
|
|
@ -280,10 +288,6 @@ which holds the transport used (RemoteTransport) and the outbound address that i
|
|||
To intercept when an outbound client is shut down you listen to ``RemoteClientShutdown``
|
||||
which holds the transport used (RemoteTransport) and the outbound address that it was connected to (Address).
|
||||
|
||||
To intercept when an outbound message cannot be sent, you listen to ``RemoteClientWriteFailed`` which holds
|
||||
the payload that was not written (AnyRef), the cause of the failed send (Throwable),
|
||||
the transport used (RemoteTransport) and the outbound address that was the destination (Address).
|
||||
|
||||
For general outbound-related errors, that do not classify as any of the others, you can listen to ``RemoteClientError``,
|
||||
which holds the cause (Throwable), the transport used (RemoteTransport) and the outbound address (Address).
|
||||
|
||||
|
|
|
|||
|
|
@ -194,10 +194,10 @@ 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 TestKit is accessible with
|
||||
the the :obj:`system` member.
|
||||
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.
|
||||
The ActorSystem passed in to the constructor of TestKit is accessible via the
|
||||
:obj:`system` member. 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
|
||||
-------------------
|
||||
|
|
@ -671,6 +671,25 @@ This section contains a collection of known gotchas with some other frameworks,
|
|||
which is by no means exhaustive and does not imply endorsement or special
|
||||
support.
|
||||
|
||||
When you need it to be a trait
|
||||
------------------------------
|
||||
|
||||
If for some reason it is a problem to inherit from :class:`TestKit` due to it
|
||||
being a concrete class instead of a trait, there’s :class:`TestKitBase`:
|
||||
|
||||
.. includecode:: code/docs/testkit/TestkitDocSpec.scala
|
||||
:include: test-kit-base
|
||||
:exclude: put-your-test-code-here
|
||||
|
||||
The ``implicit lazy val system`` must be declared exactly like that (you can of
|
||||
course pass arguments to the actor system factory as needed) because trait
|
||||
:class:`TestKitBase` needs the system during its construction.
|
||||
|
||||
.. warning::
|
||||
|
||||
Use of the trait is discouraged because of potential issues with binary
|
||||
backwards compatibility in the future, use at own risk.
|
||||
|
||||
Specs2
|
||||
------
|
||||
|
||||
|
|
|
|||
|
|
@ -168,7 +168,8 @@ trait Conductor { this: TestConductorExt ⇒
|
|||
|
||||
/**
|
||||
* Tell the remote node to shut itself down using System.exit with the given
|
||||
* exitValue.
|
||||
* exitValue. The node will also be removed, so that the remaining nodes may still
|
||||
* pass subsequent barriers.
|
||||
*
|
||||
* @param node is the symbolic name of the node which is to be affected
|
||||
* @param exitValue is the return code which shall be given to System.exit
|
||||
|
|
@ -441,10 +442,10 @@ private[akka] class Controller(private var initialParticipants: Int, controllerP
|
|||
if (exitValueOrKill < 0) {
|
||||
// TODO: kill via SBT
|
||||
} else {
|
||||
barrier ! BarrierCoordinator.RemoveClient(node)
|
||||
nodes(node).fsm forward ToClient(TerminateMsg(exitValueOrKill))
|
||||
}
|
||||
case Remove(node) ⇒
|
||||
nodes -= node
|
||||
barrier ! BarrierCoordinator.RemoveClient(node)
|
||||
}
|
||||
case GetNodes ⇒ sender ! nodes.keys
|
||||
|
|
@ -540,8 +541,8 @@ private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoor
|
|||
|
||||
when(Waiting) {
|
||||
case Event(EnterBarrier(name), d @ Data(clients, barrier, arrived)) ⇒
|
||||
if (name != barrier || clients.find(_.fsm == sender).isEmpty) throw WrongBarrier(name, sender, d)
|
||||
val together = sender :: arrived
|
||||
if (name != barrier) throw WrongBarrier(name, sender, d)
|
||||
val together = if (clients.exists(_.fsm == sender)) sender :: arrived else arrived
|
||||
handleBarrier(d.copy(arrived = together))
|
||||
case Event(RemoveClient(name), d @ Data(clients, barrier, arrived)) ⇒
|
||||
clients find (_.name == name) match {
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ private[akka] class FailureInjector extends Actor with ActorLogging {
|
|||
ctx: Option[ChannelHandlerContext] = None,
|
||||
throttleSend: Option[SetRate] = None,
|
||||
throttleReceive: Option[SetRate] = None)
|
||||
case class Injectors(sender: ActorRef, receiver: ActorRef)
|
||||
case class Injectors(sender: ActorRef, receiver: ActorRef, known: Boolean)
|
||||
|
||||
var channels = Map[ChannelHandlerContext, Injectors]()
|
||||
var settings = Map[Address, ChannelSettings]()
|
||||
|
|
@ -37,12 +37,13 @@ private[akka] class FailureInjector extends Actor with ActorLogging {
|
|||
/**
|
||||
* Only for a NEW ctx, start ThrottleActors, prime them and update all maps.
|
||||
*/
|
||||
def ingestContextAddress(ctx: ChannelHandlerContext, addr: Address): Injectors = {
|
||||
val gen = generation.next
|
||||
def ingestContextAddress(ctx: ChannelHandlerContext, addr: Address, known: Boolean,
|
||||
snd: Option[ActorRef] = None, rcv: Option[ActorRef] = None): Injectors = {
|
||||
lazy val gen = generation.next
|
||||
val name = addr.host.get + ":" + addr.port.get
|
||||
val thrSend = context.actorOf(Props(new ThrottleActor(ctx)), name + "-snd" + gen)
|
||||
val thrRecv = context.actorOf(Props(new ThrottleActor(ctx)), name + "-rcv" + gen)
|
||||
val injectors = Injectors(thrSend, thrRecv)
|
||||
val thrSend = snd getOrElse context.actorOf(Props(new ThrottleActor(ctx)), name + "-snd" + gen)
|
||||
val thrRecv = rcv getOrElse context.actorOf(Props(new ThrottleActor(ctx)), name + "-rcv" + gen)
|
||||
val injectors = Injectors(thrSend, thrRecv, known)
|
||||
channels += ctx -> injectors
|
||||
settings += addr -> (settings get addr map {
|
||||
case c @ ChannelSettings(prevCtx, ts, tr) ⇒
|
||||
|
|
@ -134,7 +135,10 @@ private[akka] class FailureInjector extends Actor with ActorLogging {
|
|||
*/
|
||||
case s @ Send(ctx, direction, future, msg) ⇒
|
||||
channels get ctx match {
|
||||
case Some(Injectors(snd, rcv)) ⇒
|
||||
case Some(Injectors(snd, rcv, known)) ⇒
|
||||
// if the system registered with an empty name then check if we know it now
|
||||
if (!known) ChannelAddress.get(ctx.getChannel).foreach(addr ⇒
|
||||
ingestContextAddress(ctx, addr, true, Some(snd), Some(rcv)))
|
||||
if (direction includes Direction.Send) snd ! s
|
||||
if (direction includes Direction.Receive) rcv ! s
|
||||
case None ⇒
|
||||
|
|
@ -142,21 +146,24 @@ private[akka] class FailureInjector extends Actor with ActorLogging {
|
|||
ctx.getChannel.getRemoteAddress match {
|
||||
case sockAddr: InetSocketAddress ⇒
|
||||
val (ipaddr, ip, port) = (sockAddr.getAddress, sockAddr.getAddress.getHostAddress, sockAddr.getPort)
|
||||
val addr = ChannelAddress.get(ctx.getChannel) orElse {
|
||||
val (addr, known) = ChannelAddress.get(ctx.getChannel) orElse {
|
||||
settings collect { case (a @ Address("akka", _, Some(`ip`), Some(`port`)), _) ⇒ a } headOption
|
||||
} orElse {
|
||||
// only if raw IP failed, try with hostname
|
||||
val name = ipaddr.getHostName
|
||||
if (name == ip) None
|
||||
else settings collect { case (a @ Address("akka", _, Some(`name`), Some(`port`)), _) ⇒ a } headOption
|
||||
} getOrElse Address("akka", "", ip, port)
|
||||
} match {
|
||||
case Some(a) ⇒ (a, true)
|
||||
case None ⇒ (Address("akka", "", ip, port), false)
|
||||
}
|
||||
/*
|
||||
* ^- the above last resort will not match later requests directly, but be
|
||||
* picked up by retrieveTargetSettings, so that throttle ops are
|
||||
* applied to the right throttle actors, assuming that there can
|
||||
* be only one actor system per host:port.
|
||||
*/
|
||||
val inj = ingestContextAddress(ctx, addr)
|
||||
val inj = ingestContextAddress(ctx, addr, known)
|
||||
if (direction includes Direction.Send) inj.sender ! s
|
||||
if (direction includes Direction.Receive) inj.receiver ! s
|
||||
case null ⇒
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.remote.testconductor
|
||||
|
||||
import org.jboss.netty.channel.{ Channel, ChannelPipeline, ChannelPipelineFactory, ChannelUpstreamHandler, SimpleChannelUpstreamHandler, StaticChannelPipeline }
|
||||
import org.jboss.netty.channel.{ Channel, ChannelPipeline, ChannelPipelineFactory, ChannelUpstreamHandler, SimpleChannelUpstreamHandler, DefaultChannelPipeline }
|
||||
import org.jboss.netty.channel.socket.nio.{ NioClientSocketChannelFactory, NioServerSocketChannelFactory }
|
||||
import org.jboss.netty.bootstrap.{ ClientBootstrap, ServerBootstrap }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
|
||||
|
|
@ -12,6 +12,7 @@ import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder
|
|||
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.Executors
|
||||
import akka.event.Logging
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
|
|
@ -21,7 +22,9 @@ private[akka] class TestConductorPipelineFactory(handler: ChannelUpstreamHandler
|
|||
val encap = List(new LengthFieldPrepender(4), new LengthFieldBasedFrameDecoder(10000, 0, 4, 0, 4))
|
||||
val proto = List(new ProtobufEncoder, new ProtobufDecoder(TestConductorProtocol.Wrapper.getDefaultInstance))
|
||||
val msg = List(new MsgEncoder, new MsgDecoder)
|
||||
new StaticChannelPipeline(encap ::: proto ::: msg ::: handler :: Nil: _*)
|
||||
(encap ::: proto ::: msg ::: handler :: Nil).foldLeft(new DefaultChannelPipeline) {
|
||||
(pipe, handler) ⇒ pipe.addLast(Logging.simpleName(handler.getClass), handler); pipe
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,153 @@
|
|||
package akka.remote.testkit
|
||||
|
||||
import java.awt.Toolkit
|
||||
import java.awt.datatransfer.Clipboard
|
||||
import java.awt.datatransfer.ClipboardOwner
|
||||
import java.awt.datatransfer.DataFlavor
|
||||
import java.awt.datatransfer.StringSelection
|
||||
import java.awt.datatransfer.Transferable
|
||||
import java.io.BufferedReader
|
||||
import java.io.FileReader
|
||||
import java.io.FileWriter
|
||||
import java.io.InputStreamReader
|
||||
import java.io.OutputStreamWriter
|
||||
import java.io.PrintWriter
|
||||
import java.io.StringReader
|
||||
import java.io.StringWriter
|
||||
import scala.annotation.tailrec
|
||||
|
||||
/**
|
||||
* Utility to make log files from multi-node tests easier to analyze.
|
||||
* Replaces jvm names and host:port with corresponding logical role name.
|
||||
*/
|
||||
object LogRoleReplace extends ClipboardOwner {
|
||||
|
||||
/**
|
||||
* Main program. Use with 0, 1 or 2 arguments.
|
||||
*
|
||||
* When using 0 arguments it reads from standard input
|
||||
* (System.in) and writes to standard output (System.out).
|
||||
*
|
||||
* With 1 argument it reads from the file specified in the first argument
|
||||
* and writes to standard output.
|
||||
*
|
||||
* With 2 arguments it reads the file specified in the first argument
|
||||
* and writes to the file specified in the second argument.
|
||||
*
|
||||
* You can also replace the contents of the clipboard instead of using files
|
||||
* by supplying `clipboard` as argument
|
||||
*/
|
||||
def main(args: Array[String]): Unit = {
|
||||
val replacer = new LogRoleReplace
|
||||
|
||||
if (args.length == 0) {
|
||||
replacer.process(
|
||||
new BufferedReader(new InputStreamReader(System.in)),
|
||||
new PrintWriter(new OutputStreamWriter(System.out)))
|
||||
|
||||
} else if (args(0) == "clipboard") {
|
||||
val clipboard = Toolkit.getDefaultToolkit.getSystemClipboard
|
||||
val contents = clipboard.getContents(null)
|
||||
if (contents != null && contents.isDataFlavorSupported(DataFlavor.stringFlavor)) {
|
||||
val text = contents.getTransferData(DataFlavor.stringFlavor).asInstanceOf[String]
|
||||
val result = new StringWriter
|
||||
replacer.process(
|
||||
new BufferedReader(new StringReader(text)),
|
||||
new PrintWriter(result))
|
||||
clipboard.setContents(new StringSelection(result.toString), this)
|
||||
println("Replaced clipboard contents")
|
||||
}
|
||||
|
||||
} else if (args.length == 1) {
|
||||
val inputFile = new BufferedReader(new FileReader(args(0)))
|
||||
try {
|
||||
replacer.process(
|
||||
inputFile,
|
||||
new PrintWriter(new OutputStreamWriter(System.out)))
|
||||
} finally {
|
||||
inputFile.close()
|
||||
}
|
||||
|
||||
} else if (args.length == 2) {
|
||||
val outputFile = new PrintWriter(new FileWriter(args(1)))
|
||||
val inputFile = new BufferedReader(new FileReader(args(0)))
|
||||
try {
|
||||
replacer.process(inputFile, outputFile)
|
||||
} finally {
|
||||
outputFile.close()
|
||||
inputFile.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Empty implementation of the ClipboardOwner interface
|
||||
*/
|
||||
def lostOwnership(clipboard: Clipboard, contents: Transferable): Unit = ()
|
||||
}
|
||||
|
||||
class LogRoleReplace {
|
||||
|
||||
private val RoleStarted = """\[([\w\-]+)\].*Role \[([\w]+)\] started""".r
|
||||
private val RemoteServerStarted = """\[([\w\-]+)\].*RemoteServerStarted@akka://.*@([\w\-\.]+):([0-9]+)""".r
|
||||
private val ColorCode = """\[[0-9]+m"""
|
||||
|
||||
private var replacements: Map[String, String] = Map.empty
|
||||
private var jvmToAddress: Map[String, String] = Map.empty
|
||||
|
||||
def process(in: BufferedReader, out: PrintWriter): Unit = {
|
||||
|
||||
@tailrec
|
||||
def processLines(line: String): Unit = if (line ne null) {
|
||||
out.println(processLine(line))
|
||||
processLines(in.readLine)
|
||||
}
|
||||
|
||||
processLines(in.readLine())
|
||||
}
|
||||
|
||||
def processLine(line: String): String = {
|
||||
val cleanLine = removeColorCodes(line)
|
||||
if (updateReplacements(cleanLine))
|
||||
replaceLine(cleanLine)
|
||||
else
|
||||
cleanLine
|
||||
}
|
||||
|
||||
private def removeColorCodes(line: String): String =
|
||||
line.replaceAll(ColorCode, "")
|
||||
|
||||
private def updateReplacements(line: String): Boolean = {
|
||||
if (line.startsWith("[info] * ")) {
|
||||
// reset when new test begins
|
||||
replacements = Map.empty
|
||||
jvmToAddress = Map.empty
|
||||
}
|
||||
|
||||
line match {
|
||||
case RemoteServerStarted(jvm, host, port) ⇒
|
||||
jvmToAddress += (jvm -> (host + ":" + port))
|
||||
false
|
||||
|
||||
case RoleStarted(jvm, role) ⇒
|
||||
jvmToAddress.get(jvm) match {
|
||||
case Some(address) ⇒
|
||||
replacements += (jvm -> role)
|
||||
replacements += (address -> role)
|
||||
false
|
||||
case None ⇒ false
|
||||
}
|
||||
|
||||
case _ ⇒ true
|
||||
}
|
||||
}
|
||||
|
||||
private def replaceLine(line: String): String = {
|
||||
var result = line
|
||||
for ((from, to) ← replacements) {
|
||||
result = result.replaceAll(from, to)
|
||||
}
|
||||
result
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -3,23 +3,16 @@
|
|||
*/
|
||||
package akka.remote.testkit
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.{ ActorSystem, ExtendedActorSystem }
|
||||
import akka.remote.testconductor.TestConductor
|
||||
import java.net.InetAddress
|
||||
import java.net.InetSocketAddress
|
||||
import akka.remote.testconductor.TestConductorExt
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.dispatch.Await.Awaitable
|
||||
|
||||
import com.typesafe.config.{ ConfigObject, ConfigFactory, Config }
|
||||
|
||||
import akka.actor.{ RootActorPath, Deploy, ActorPath, ActorSystem, ExtendedActorSystem }
|
||||
import akka.dispatch.Await
|
||||
import akka.util.Duration
|
||||
import akka.util.NonFatal
|
||||
import akka.actor.ActorPath
|
||||
import akka.actor.RootActorPath
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.actor.Deploy
|
||||
import com.typesafe.config.ConfigObject
|
||||
import akka.dispatch.Await.Awaitable
|
||||
import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName }
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.util.{ NonFatal, Duration }
|
||||
|
||||
/**
|
||||
* Configure the role names and participants of the test, including configuration settings.
|
||||
|
|
@ -138,18 +131,23 @@ object MultiNodeSpec {
|
|||
* `AskTimeoutException: sending to terminated ref breaks promises`. Using lazy
|
||||
* val is fine.
|
||||
*/
|
||||
abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, roles: Seq[RoleName], deployments: RoleName ⇒ Seq[String])
|
||||
abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: Seq[RoleName], deployments: RoleName ⇒ Seq[String])
|
||||
extends AkkaSpec(_system) {
|
||||
|
||||
import MultiNodeSpec._
|
||||
|
||||
def this(config: MultiNodeConfig) =
|
||||
this(config.myself, ActorSystem(AkkaSpec.getCallerName, config.config), config.roles, config.deployments)
|
||||
this(config.myself, ActorSystem(AkkaSpec.getCallerName(classOf[MultiNodeSpec]), config.config), config.roles, config.deployments)
|
||||
|
||||
/*
|
||||
* Test Class Interface
|
||||
*/
|
||||
|
||||
/**
|
||||
* All registered roles
|
||||
*/
|
||||
def roles: Seq[RoleName] = _roles
|
||||
|
||||
/**
|
||||
* TO BE DEFINED BY USER: Defines the number of participants required for starting the test. This
|
||||
* might not be equals to the number of nodes available to the test.
|
||||
|
|
@ -249,4 +247,7 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, roles:
|
|||
}
|
||||
}
|
||||
|
||||
}
|
||||
// useful to see which jvm is running which role
|
||||
log.info("Role [{}] started", myself.name)
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6351,605 +6351,6 @@ public final class RemoteProtocol {
|
|||
// @@protoc_insertion_point(class_scope:DeployProtocol)
|
||||
}
|
||||
|
||||
public interface DaemonMsgWatchProtocolOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// required .ActorRefProtocol watcher = 1;
|
||||
boolean hasWatcher();
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol getWatcher();
|
||||
akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatcherOrBuilder();
|
||||
|
||||
// required .ActorRefProtocol watched = 2;
|
||||
boolean hasWatched();
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol getWatched();
|
||||
akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatchedOrBuilder();
|
||||
}
|
||||
public static final class DaemonMsgWatchProtocol extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
implements DaemonMsgWatchProtocolOrBuilder {
|
||||
// Use DaemonMsgWatchProtocol.newBuilder() to construct.
|
||||
private DaemonMsgWatchProtocol(Builder builder) {
|
||||
super(builder);
|
||||
}
|
||||
private DaemonMsgWatchProtocol(boolean noInit) {}
|
||||
|
||||
private static final DaemonMsgWatchProtocol defaultInstance;
|
||||
public static DaemonMsgWatchProtocol getDefaultInstance() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public DaemonMsgWatchProtocol getDefaultInstanceForType() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_fieldAccessorTable;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// required .ActorRefProtocol watcher = 1;
|
||||
public static final int WATCHER_FIELD_NUMBER = 1;
|
||||
private akka.remote.RemoteProtocol.ActorRefProtocol watcher_;
|
||||
public boolean hasWatcher() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocol getWatcher() {
|
||||
return watcher_;
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatcherOrBuilder() {
|
||||
return watcher_;
|
||||
}
|
||||
|
||||
// required .ActorRefProtocol watched = 2;
|
||||
public static final int WATCHED_FIELD_NUMBER = 2;
|
||||
private akka.remote.RemoteProtocol.ActorRefProtocol watched_;
|
||||
public boolean hasWatched() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocol getWatched() {
|
||||
return watched_;
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatchedOrBuilder() {
|
||||
return watched_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
if (isInitialized != -1) return isInitialized == 1;
|
||||
|
||||
if (!hasWatcher()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
if (!hasWatched()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
if (!getWatcher().isInitialized()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
if (!getWatched().isInitialized()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
memoizedIsInitialized = 1;
|
||||
return true;
|
||||
}
|
||||
|
||||
public void writeTo(com.google.protobuf.CodedOutputStream output)
|
||||
throws java.io.IOException {
|
||||
getSerializedSize();
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeMessage(1, watcher_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeMessage(2, watched_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
private int memoizedSerializedSize = -1;
|
||||
public int getSerializedSize() {
|
||||
int size = memoizedSerializedSize;
|
||||
if (size != -1) return size;
|
||||
|
||||
size = 0;
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(1, watcher_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(2, watched_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
}
|
||||
|
||||
private static final long serialVersionUID = 0L;
|
||||
@java.lang.Override
|
||||
protected java.lang.Object writeReplace()
|
||||
throws java.io.ObjectStreamException {
|
||||
return super.writeReplace();
|
||||
}
|
||||
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
|
||||
com.google.protobuf.ByteString data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
|
||||
com.google.protobuf.ByteString data,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(byte[] data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
|
||||
byte[] data,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
|
||||
java.io.InputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseDelimitedFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
Builder builder = newBuilder();
|
||||
if (builder.mergeDelimitedFrom(input)) {
|
||||
return builder.buildParsed();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseDelimitedFrom(
|
||||
java.io.InputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
Builder builder = newBuilder();
|
||||
if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
|
||||
return builder.buildParsed();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
|
||||
com.google.protobuf.CodedInputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static akka.remote.RemoteProtocol.DaemonMsgWatchProtocol parseFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input, extensionRegistry)
|
||||
.buildParsed();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() { return Builder.create(); }
|
||||
public Builder newBuilderForType() { return newBuilder(); }
|
||||
public static Builder newBuilder(akka.remote.RemoteProtocol.DaemonMsgWatchProtocol prototype) {
|
||||
return newBuilder().mergeFrom(prototype);
|
||||
}
|
||||
public Builder toBuilder() { return newBuilder(this); }
|
||||
|
||||
@java.lang.Override
|
||||
protected Builder newBuilderForType(
|
||||
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
Builder builder = new Builder(parent);
|
||||
return builder;
|
||||
}
|
||||
public static final class Builder extends
|
||||
com.google.protobuf.GeneratedMessage.Builder<Builder>
|
||||
implements akka.remote.RemoteProtocol.DaemonMsgWatchProtocolOrBuilder {
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return akka.remote.RemoteProtocol.internal_static_DaemonMsgWatchProtocol_fieldAccessorTable;
|
||||
}
|
||||
|
||||
// Construct using akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.newBuilder()
|
||||
private Builder() {
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
|
||||
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||
super(parent);
|
||||
maybeForceBuilderInitialization();
|
||||
}
|
||||
private void maybeForceBuilderInitialization() {
|
||||
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||
getWatcherFieldBuilder();
|
||||
getWatchedFieldBuilder();
|
||||
}
|
||||
}
|
||||
private static Builder create() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public Builder clear() {
|
||||
super.clear();
|
||||
if (watcherBuilder_ == null) {
|
||||
watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
} else {
|
||||
watcherBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
if (watchedBuilder_ == null) {
|
||||
watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
} else {
|
||||
watchedBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder clone() {
|
||||
return create().mergeFrom(buildPartial());
|
||||
}
|
||||
|
||||
public com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptorForType() {
|
||||
return akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.getDescriptor();
|
||||
}
|
||||
|
||||
public akka.remote.RemoteProtocol.DaemonMsgWatchProtocol getDefaultInstanceForType() {
|
||||
return akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.getDefaultInstance();
|
||||
}
|
||||
|
||||
public akka.remote.RemoteProtocol.DaemonMsgWatchProtocol build() {
|
||||
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private akka.remote.RemoteProtocol.DaemonMsgWatchProtocol buildParsed()
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(
|
||||
result).asInvalidProtocolBufferException();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public akka.remote.RemoteProtocol.DaemonMsgWatchProtocol buildPartial() {
|
||||
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol result = new akka.remote.RemoteProtocol.DaemonMsgWatchProtocol(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
if (watcherBuilder_ == null) {
|
||||
result.watcher_ = watcher_;
|
||||
} else {
|
||||
result.watcher_ = watcherBuilder_.build();
|
||||
}
|
||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
if (watchedBuilder_ == null) {
|
||||
result.watched_ = watched_;
|
||||
} else {
|
||||
result.watched_ = watchedBuilder_.build();
|
||||
}
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||
if (other instanceof akka.remote.RemoteProtocol.DaemonMsgWatchProtocol) {
|
||||
return mergeFrom((akka.remote.RemoteProtocol.DaemonMsgWatchProtocol)other);
|
||||
} else {
|
||||
super.mergeFrom(other);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder mergeFrom(akka.remote.RemoteProtocol.DaemonMsgWatchProtocol other) {
|
||||
if (other == akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.getDefaultInstance()) return this;
|
||||
if (other.hasWatcher()) {
|
||||
mergeWatcher(other.getWatcher());
|
||||
}
|
||||
if (other.hasWatched()) {
|
||||
mergeWatched(other.getWatched());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
||||
public final boolean isInitialized() {
|
||||
if (!hasWatcher()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
if (!hasWatched()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
if (!getWatcher().isInitialized()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
if (!getWatched().isInitialized()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public Builder mergeFrom(
|
||||
com.google.protobuf.CodedInputStream input,
|
||||
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||
throws java.io.IOException {
|
||||
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
|
||||
com.google.protobuf.UnknownFieldSet.newBuilder(
|
||||
this.getUnknownFields());
|
||||
while (true) {
|
||||
int tag = input.readTag();
|
||||
switch (tag) {
|
||||
case 0:
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
default: {
|
||||
if (!parseUnknownField(input, unknownFields,
|
||||
extensionRegistry, tag)) {
|
||||
this.setUnknownFields(unknownFields.build());
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 10: {
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder();
|
||||
if (hasWatcher()) {
|
||||
subBuilder.mergeFrom(getWatcher());
|
||||
}
|
||||
input.readMessage(subBuilder, extensionRegistry);
|
||||
setWatcher(subBuilder.buildPartial());
|
||||
break;
|
||||
}
|
||||
case 18: {
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder();
|
||||
if (hasWatched()) {
|
||||
subBuilder.mergeFrom(getWatched());
|
||||
}
|
||||
input.readMessage(subBuilder, extensionRegistry);
|
||||
setWatched(subBuilder.buildPartial());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
|
||||
// required .ActorRefProtocol watcher = 1;
|
||||
private akka.remote.RemoteProtocol.ActorRefProtocol watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
private com.google.protobuf.SingleFieldBuilder<
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> watcherBuilder_;
|
||||
public boolean hasWatcher() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocol getWatcher() {
|
||||
if (watcherBuilder_ == null) {
|
||||
return watcher_;
|
||||
} else {
|
||||
return watcherBuilder_.getMessage();
|
||||
}
|
||||
}
|
||||
public Builder setWatcher(akka.remote.RemoteProtocol.ActorRefProtocol value) {
|
||||
if (watcherBuilder_ == null) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
watcher_ = value;
|
||||
onChanged();
|
||||
} else {
|
||||
watcherBuilder_.setMessage(value);
|
||||
}
|
||||
bitField0_ |= 0x00000001;
|
||||
return this;
|
||||
}
|
||||
public Builder setWatcher(
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) {
|
||||
if (watcherBuilder_ == null) {
|
||||
watcher_ = builderForValue.build();
|
||||
onChanged();
|
||||
} else {
|
||||
watcherBuilder_.setMessage(builderForValue.build());
|
||||
}
|
||||
bitField0_ |= 0x00000001;
|
||||
return this;
|
||||
}
|
||||
public Builder mergeWatcher(akka.remote.RemoteProtocol.ActorRefProtocol value) {
|
||||
if (watcherBuilder_ == null) {
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001) &&
|
||||
watcher_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) {
|
||||
watcher_ =
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(watcher_).mergeFrom(value).buildPartial();
|
||||
} else {
|
||||
watcher_ = value;
|
||||
}
|
||||
onChanged();
|
||||
} else {
|
||||
watcherBuilder_.mergeFrom(value);
|
||||
}
|
||||
bitField0_ |= 0x00000001;
|
||||
return this;
|
||||
}
|
||||
public Builder clearWatcher() {
|
||||
if (watcherBuilder_ == null) {
|
||||
watcher_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
onChanged();
|
||||
} else {
|
||||
watcherBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
return this;
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getWatcherBuilder() {
|
||||
bitField0_ |= 0x00000001;
|
||||
onChanged();
|
||||
return getWatcherFieldBuilder().getBuilder();
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatcherOrBuilder() {
|
||||
if (watcherBuilder_ != null) {
|
||||
return watcherBuilder_.getMessageOrBuilder();
|
||||
} else {
|
||||
return watcher_;
|
||||
}
|
||||
}
|
||||
private com.google.protobuf.SingleFieldBuilder<
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>
|
||||
getWatcherFieldBuilder() {
|
||||
if (watcherBuilder_ == null) {
|
||||
watcherBuilder_ = new com.google.protobuf.SingleFieldBuilder<
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>(
|
||||
watcher_,
|
||||
getParentForChildren(),
|
||||
isClean());
|
||||
watcher_ = null;
|
||||
}
|
||||
return watcherBuilder_;
|
||||
}
|
||||
|
||||
// required .ActorRefProtocol watched = 2;
|
||||
private akka.remote.RemoteProtocol.ActorRefProtocol watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
private com.google.protobuf.SingleFieldBuilder<
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> watchedBuilder_;
|
||||
public boolean hasWatched() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocol getWatched() {
|
||||
if (watchedBuilder_ == null) {
|
||||
return watched_;
|
||||
} else {
|
||||
return watchedBuilder_.getMessage();
|
||||
}
|
||||
}
|
||||
public Builder setWatched(akka.remote.RemoteProtocol.ActorRefProtocol value) {
|
||||
if (watchedBuilder_ == null) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
watched_ = value;
|
||||
onChanged();
|
||||
} else {
|
||||
watchedBuilder_.setMessage(value);
|
||||
}
|
||||
bitField0_ |= 0x00000002;
|
||||
return this;
|
||||
}
|
||||
public Builder setWatched(
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) {
|
||||
if (watchedBuilder_ == null) {
|
||||
watched_ = builderForValue.build();
|
||||
onChanged();
|
||||
} else {
|
||||
watchedBuilder_.setMessage(builderForValue.build());
|
||||
}
|
||||
bitField0_ |= 0x00000002;
|
||||
return this;
|
||||
}
|
||||
public Builder mergeWatched(akka.remote.RemoteProtocol.ActorRefProtocol value) {
|
||||
if (watchedBuilder_ == null) {
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002) &&
|
||||
watched_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) {
|
||||
watched_ =
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(watched_).mergeFrom(value).buildPartial();
|
||||
} else {
|
||||
watched_ = value;
|
||||
}
|
||||
onChanged();
|
||||
} else {
|
||||
watchedBuilder_.mergeFrom(value);
|
||||
}
|
||||
bitField0_ |= 0x00000002;
|
||||
return this;
|
||||
}
|
||||
public Builder clearWatched() {
|
||||
if (watchedBuilder_ == null) {
|
||||
watched_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance();
|
||||
onChanged();
|
||||
} else {
|
||||
watchedBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
return this;
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getWatchedBuilder() {
|
||||
bitField0_ |= 0x00000002;
|
||||
onChanged();
|
||||
return getWatchedFieldBuilder().getBuilder();
|
||||
}
|
||||
public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getWatchedOrBuilder() {
|
||||
if (watchedBuilder_ != null) {
|
||||
return watchedBuilder_.getMessageOrBuilder();
|
||||
} else {
|
||||
return watched_;
|
||||
}
|
||||
}
|
||||
private com.google.protobuf.SingleFieldBuilder<
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>
|
||||
getWatchedFieldBuilder() {
|
||||
if (watchedBuilder_ == null) {
|
||||
watchedBuilder_ = new com.google.protobuf.SingleFieldBuilder<
|
||||
akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>(
|
||||
watched_,
|
||||
getParentForChildren(),
|
||||
isClean());
|
||||
watched_ = null;
|
||||
}
|
||||
return watchedBuilder_;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:DaemonMsgWatchProtocol)
|
||||
}
|
||||
|
||||
static {
|
||||
defaultInstance = new DaemonMsgWatchProtocol(true);
|
||||
defaultInstance.initFields();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(class_scope:DaemonMsgWatchProtocol)
|
||||
}
|
||||
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_AkkaRemoteProtocol_descriptor;
|
||||
private static
|
||||
|
|
@ -7000,11 +6401,6 @@ public final class RemoteProtocol {
|
|||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_DeployProtocol_fieldAccessorTable;
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_DaemonMsgWatchProtocol_descriptor;
|
||||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_DaemonMsgWatchProtocol_fieldAccessorTable;
|
||||
|
||||
public static com.google.protobuf.Descriptors.FileDescriptor
|
||||
getDescriptor() {
|
||||
|
|
@ -7039,11 +6435,9 @@ public final class RemoteProtocol {
|
|||
"ssCreator\030\003 \001(\t\022\017\n\007creator\030\004 \001(\014\022\024\n\014rout" +
|
||||
"erConfig\030\005 \001(\014\"S\n\016DeployProtocol\022\014\n\004path" +
|
||||
"\030\001 \002(\t\022\016\n\006config\030\002 \001(\014\022\024\n\014routerConfig\030\003" +
|
||||
" \001(\014\022\r\n\005scope\030\004 \001(\014\"`\n\026DaemonMsgWatchPro" +
|
||||
"tocol\022\"\n\007watcher\030\001 \002(\0132\021.ActorRefProtoco" +
|
||||
"l\022\"\n\007watched\030\002 \002(\0132\021.ActorRefProtocol*7\n" +
|
||||
"\013CommandType\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOWN\020\002\022" +
|
||||
"\r\n\tHEARTBEAT\020\003B\017\n\013akka.remoteH\001"
|
||||
" \001(\014\022\r\n\005scope\030\004 \001(\014*7\n\013CommandType\022\013\n\007CO" +
|
||||
"NNECT\020\001\022\014\n\010SHUTDOWN\020\002\022\r\n\tHEARTBEAT\020\003B\017\n\013" +
|
||||
"akka.remoteH\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
|
@ -7130,14 +6524,6 @@ public final class RemoteProtocol {
|
|||
new java.lang.String[] { "Path", "Config", "RouterConfig", "Scope", },
|
||||
akka.remote.RemoteProtocol.DeployProtocol.class,
|
||||
akka.remote.RemoteProtocol.DeployProtocol.Builder.class);
|
||||
internal_static_DaemonMsgWatchProtocol_descriptor =
|
||||
getDescriptor().getMessageTypes().get(10);
|
||||
internal_static_DaemonMsgWatchProtocol_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_DaemonMsgWatchProtocol_descriptor,
|
||||
new java.lang.String[] { "Watcher", "Watched", },
|
||||
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.class,
|
||||
akka.remote.RemoteProtocol.DaemonMsgWatchProtocol.Builder.class);
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
|
|
|||
|
|
@ -107,12 +107,4 @@ message DeployProtocol {
|
|||
optional bytes config = 2;
|
||||
optional bytes routerConfig = 3;
|
||||
optional bytes scope = 4;
|
||||
}
|
||||
|
||||
/**
|
||||
* Serialization of akka.remote.DaemonMsgWatch
|
||||
*/
|
||||
message DaemonMsgWatchProtocol {
|
||||
required ActorRefProtocol watcher = 1;
|
||||
required ActorRefProtocol watched = 2;
|
||||
}
|
||||
}
|
||||
|
|
@ -15,7 +15,6 @@ akka {
|
|||
serializers {
|
||||
proto = "akka.serialization.ProtobufSerializer"
|
||||
daemon-create = "akka.serialization.DaemonMsgCreateSerializer"
|
||||
daemon-watch = "akka.serialization.DaemonMsgWatchSerializer"
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -24,7 +23,6 @@ akka {
|
|||
# does, need to use the more specific one here in order to avoid ambiguity
|
||||
"com.google.protobuf.GeneratedMessage" = proto
|
||||
"akka.remote.DaemonMsgCreate" = daemon-create
|
||||
"akka.remote.DaemonMsgWatch" = daemon-watch
|
||||
}
|
||||
|
||||
deployment {
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.remote
|
|||
|
||||
import akka.actor._
|
||||
import akka.dispatch._
|
||||
import akka.event.{ DeathWatch, Logging, LoggingAdapter }
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.event.EventStream
|
||||
import akka.serialization.Serialization
|
||||
import akka.serialization.SerializationExtension
|
||||
|
|
@ -34,8 +34,6 @@ private[akka] class RemoteActorRefProvider(
|
|||
override def rootPath: ActorPath = local.rootPath
|
||||
override def deadLetters: InternalActorRef = local.deadLetters
|
||||
|
||||
override val deathWatch: DeathWatch = new RemoteDeathWatch(local.deathWatch, this)
|
||||
|
||||
// these are only available after init()
|
||||
override def rootGuardian: InternalActorRef = local.rootGuardian
|
||||
override def guardian: InternalActorRef = local.guardian
|
||||
|
|
@ -246,25 +244,4 @@ private[akka] class RemoteActorRef private[akka] (
|
|||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
private[akka] class RemoteDeathWatch(val local: DeathWatch, val provider: RemoteActorRefProvider) extends DeathWatch {
|
||||
|
||||
override def subscribe(watcher: ActorRef, watched: ActorRef): Boolean = watched match {
|
||||
case r: RemoteRef ⇒
|
||||
val ret = local.subscribe(watcher, watched)
|
||||
provider.actorFor(r.path.root / "remote") ! DaemonMsgWatch(watcher, watched)
|
||||
ret
|
||||
case l: LocalRef ⇒
|
||||
local.subscribe(watcher, watched)
|
||||
case _ ⇒
|
||||
provider.log.error("unknown ActorRef type {} as DeathWatch target", watched.getClass)
|
||||
false
|
||||
}
|
||||
|
||||
override def unsubscribe(watcher: ActorRef, watched: ActorRef): Boolean = local.unsubscribe(watcher, watched)
|
||||
|
||||
override def unsubscribe(watcher: ActorRef): Unit = local.unsubscribe(watcher)
|
||||
|
||||
override def publish(event: Terminated): Unit = local.publish(event)
|
||||
}
|
||||
}
|
||||
|
|
@ -8,10 +8,10 @@ import scala.annotation.tailrec
|
|||
|
||||
import akka.actor.{ VirtualPathContainer, Terminated, Deploy, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor }
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.dispatch.Watch
|
||||
|
||||
private[akka] sealed trait DaemonMsg
|
||||
private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: String, supervisor: ActorRef) extends DaemonMsg
|
||||
private[akka] case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg
|
||||
|
||||
/**
|
||||
* Internal system "daemon" actor for remote internal communication.
|
||||
|
|
@ -62,18 +62,15 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath
|
|||
val actor = system.provider.actorOf(system, props, supervisor.asInstanceOf[InternalActorRef],
|
||||
path, false, Some(deploy), true)
|
||||
addChild(subpath.mkString("/"), actor)
|
||||
system.deathWatch.subscribe(this, actor)
|
||||
this.sendSystemMessage(Watch(actor, this))
|
||||
case _ ⇒
|
||||
log.error("remote path does not match path from message [{}]", message)
|
||||
}
|
||||
case DaemonMsgWatch(watcher, watched) ⇒
|
||||
val other = system.actorFor(watcher.path.root / "remote")
|
||||
system.deathWatch.subscribe(other, watched)
|
||||
}
|
||||
|
||||
case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/"))
|
||||
|
||||
case t: Terminated ⇒ system.deathWatch.publish(t)
|
||||
case t: Terminated ⇒
|
||||
|
||||
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -77,21 +77,6 @@ case class RemoteClientShutdown(
|
|||
override def toString: String = "RemoteClientShutdown@" + remoteAddress
|
||||
}
|
||||
|
||||
/**
|
||||
* RemoteClientWriteFailed is published when a remote send of a message detectably fails (throws an exception).
|
||||
*/
|
||||
case class RemoteClientWriteFailed(
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@transient @BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel: Logging.LogLevel = Logging.WarningLevel
|
||||
override def toString: String =
|
||||
"RemoteClientWriteFailed@" + remoteAddress +
|
||||
": MessageClass[" + (if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" + cause + "]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteServer.
|
||||
*/
|
||||
|
|
@ -287,10 +272,9 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re
|
|||
case l: LocalRef ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
|
||||
remoteMessage.payload match {
|
||||
case _: SystemMessage if useUntrustedMode ⇒ log.warning("operating in UntrustedMode, dropping inbound system message")
|
||||
case _: AutoReceivedMessage if useUntrustedMode ⇒ log.warning("operating in UntrustedMode, dropping inbound AutoReceivedMessage")
|
||||
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
||||
case msg ⇒ l.!(msg)(remoteMessage.sender)
|
||||
case msg: PossiblyHarmful if useUntrustedMode ⇒ log.warning("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
|
||||
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
||||
case msg ⇒ l.!(msg)(remoteMessage.sender)
|
||||
}
|
||||
case r: RemoteRef ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage)
|
||||
|
|
|
|||
|
|
@ -8,16 +8,16 @@ import java.net.{ InetAddress, InetSocketAddress }
|
|||
import org.jboss.netty.util.{ Timeout, TimerTask, HashedWheelTimer }
|
||||
import org.jboss.netty.bootstrap.ClientBootstrap
|
||||
import org.jboss.netty.channel.group.DefaultChannelGroup
|
||||
import org.jboss.netty.channel.{ ChannelFutureListener, ChannelHandler, StaticChannelPipeline, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel }
|
||||
import org.jboss.netty.channel.{ ChannelFutureListener, ChannelHandler, DefaultChannelPipeline, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.execution.ExecutionHandler
|
||||
import org.jboss.netty.handler.timeout.{ IdleState, IdleStateEvent, IdleStateAwareChannelHandler, IdleStateHandler }
|
||||
import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol }
|
||||
import akka.remote.{ RemoteProtocol, RemoteMessage, RemoteLifeCycleEvent, RemoteClientStarted, RemoteClientShutdown, RemoteClientException, RemoteClientError, RemoteClientDisconnected, RemoteClientConnected, RemoteClientWriteFailed }
|
||||
import akka.actor.{ Address, ActorRef }
|
||||
import akka.remote.{ RemoteProtocol, RemoteMessage, RemoteLifeCycleEvent, RemoteClientStarted, RemoteClientShutdown, RemoteClientException, RemoteClientError, RemoteClientDisconnected, RemoteClientConnected }
|
||||
import akka.AkkaException
|
||||
import akka.event.Logging
|
||||
import akka.util.Switch
|
||||
import akka.actor.{ DeadLetter, Address, ActorRef }
|
||||
import akka.util.{ NonFatal, Switch }
|
||||
|
||||
/**
|
||||
* This is the abstract baseclass for netty remote clients, currently there's only an
|
||||
|
|
@ -63,11 +63,13 @@ private[akka] abstract class RemoteClient private[akka] (val netty: NettyRemoteT
|
|||
val f = channel.write(request)
|
||||
f.addListener(
|
||||
new ChannelFutureListener {
|
||||
def operationComplete(future: ChannelFuture) {
|
||||
if (future.isCancelled || !future.isSuccess) {
|
||||
netty.notifyListeners(RemoteClientWriteFailed(request, future.getCause, netty, remoteAddress))
|
||||
import netty.system.deadLetters
|
||||
def operationComplete(future: ChannelFuture): Unit =
|
||||
if (future.isCancelled || !future.isSuccess) request match {
|
||||
case (msg, sender, recipient) ⇒ deadLetters ! DeadLetter(msg, sender.getOrElse(deadLetters), recipient)
|
||||
// We don't call notifyListeners here since we don't think failed message deliveries are errors
|
||||
/// If the connection goes down we'll get the error reporting done by the pipeline.
|
||||
}
|
||||
}
|
||||
})
|
||||
// Check if we should back off
|
||||
if (!channel.isWritable) {
|
||||
|
|
@ -75,11 +77,11 @@ private[akka] abstract class RemoteClient private[akka] (val netty: NettyRemoteT
|
|||
if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress))
|
||||
case NonFatal(e) ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
override def toString: String = name
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import java.util.concurrent.Executors
|
|||
import scala.collection.mutable.HashMap
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
|
||||
import org.jboss.netty.channel.{ ChannelHandlerContext, Channel, StaticChannelPipeline, ChannelHandler, ChannelPipelineFactory, ChannelLocal }
|
||||
import org.jboss.netty.channel.{ ChannelHandlerContext, Channel, DefaultChannelPipeline, ChannelHandler, ChannelPipelineFactory, ChannelLocal }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder }
|
||||
import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor }
|
||||
|
|
@ -50,10 +50,13 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
*/
|
||||
object PipelineFactory {
|
||||
/**
|
||||
* Construct a StaticChannelPipeline from a sequence of handlers; to be used
|
||||
* Construct a DefaultChannelPipeline from a sequence of handlers; to be used
|
||||
* in implementations of ChannelPipelineFactory.
|
||||
*/
|
||||
def apply(handlers: Seq[ChannelHandler]): StaticChannelPipeline = new StaticChannelPipeline(handlers: _*)
|
||||
def apply(handlers: Seq[ChannelHandler]): DefaultChannelPipeline =
|
||||
handlers.foldLeft(new DefaultChannelPipeline) {
|
||||
(pipe, handler) ⇒ pipe.addLast(Logging.simpleName(handler.getClass), handler); pipe
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs the NettyRemoteTransport default pipeline with the give “head” handler, which
|
||||
|
|
|
|||
|
|
@ -1,43 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.serialization
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.remote.DaemonMsgWatch
|
||||
import akka.remote.RemoteProtocol.ActorRefProtocol
|
||||
import akka.remote.RemoteProtocol.DaemonMsgWatchProtocol
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
||||
/**
|
||||
* Serializes akka's internal DaemonMsgWatch using protobuf.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class DaemonMsgWatchSerializer(val system: ExtendedActorSystem) extends Serializer {
|
||||
import ProtobufSerializer.serializeActorRef
|
||||
import ProtobufSerializer.deserializeActorRef
|
||||
|
||||
def includeManifest: Boolean = false
|
||||
def identifier = 4
|
||||
|
||||
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
||||
case DaemonMsgWatch(watcher, watched) ⇒
|
||||
DaemonMsgWatchProtocol.newBuilder.
|
||||
setWatcher(serializeActorRef(watcher)).
|
||||
setWatched(serializeActorRef(watched)).
|
||||
build.toByteArray
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException(
|
||||
"Can't serialize a non-DaemonMsgWatch message using DaemonMsgWatchSerializer [%s]".format(obj))
|
||||
}
|
||||
|
||||
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
|
||||
val proto = DaemonMsgWatchProtocol.parseFrom(bytes)
|
||||
DaemonMsgWatch(
|
||||
watcher = deserializeActorRef(system, proto.getWatcher),
|
||||
watched = deserializeActorRef(system, proto.getWatched))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -27,7 +27,7 @@ object ProtobufSerializer {
|
|||
/**
|
||||
* Helper to materialize (lookup) an [[akka.actor.ActorRef]]
|
||||
* from Akka's protobuf representation in the supplied
|
||||
* [[akka.actor.ActorSystem].
|
||||
* [[akka.actor.ActorSystem]].
|
||||
*/
|
||||
def deserializeActorRef(system: ActorSystem, refProtocol: ActorRefProtocol): ActorRef =
|
||||
system.actorFor(refProtocol.getPath)
|
||||
|
|
|
|||
|
|
@ -1,49 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.serialization
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.remote.DaemonMsgWatch
|
||||
import akka.actor.Actor
|
||||
import akka.actor.Props
|
||||
|
||||
object DaemonMsgWatchSerializerSpec {
|
||||
class MyActor extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class DaemonMsgWatchSerializerSpec extends AkkaSpec {
|
||||
|
||||
import DaemonMsgWatchSerializerSpec._
|
||||
|
||||
val ser = SerializationExtension(system)
|
||||
|
||||
"Serialization" must {
|
||||
|
||||
"resolve DaemonMsgWatchSerializer" in {
|
||||
ser.serializerFor(classOf[DaemonMsgWatch]).getClass must be(classOf[DaemonMsgWatchSerializer])
|
||||
}
|
||||
|
||||
"serialize and de-serialize DaemonMsgWatch" in {
|
||||
val watcher = system.actorOf(Props[MyActor], "watcher")
|
||||
val watched = system.actorOf(Props[MyActor], "watched")
|
||||
val msg = DaemonMsgWatch(watcher, watched)
|
||||
val bytes = ser.serialize(msg) match {
|
||||
case Left(exception) ⇒ fail(exception)
|
||||
case Right(bytes) ⇒ bytes
|
||||
}
|
||||
ser.deserialize(bytes.asInstanceOf[Array[Byte]], classOf[DaemonMsgWatch]) match {
|
||||
case Left(exception) ⇒ fail(exception)
|
||||
case Right(m) ⇒ assert(m === msg)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -31,18 +31,12 @@ class TestActorRef[T <: Actor](
|
|||
if (_props.dispatcher == Dispatchers.DefaultDispatcherId) CallingThreadDispatcher.Id
|
||||
else _props.dispatcher),
|
||||
_supervisor,
|
||||
_supervisor.path / name,
|
||||
false) {
|
||||
_supervisor.path / name) {
|
||||
|
||||
import TestActorRef.InternalGetActor
|
||||
|
||||
override def newActorCell(
|
||||
system: ActorSystemImpl,
|
||||
ref: InternalActorRef,
|
||||
props: Props,
|
||||
supervisor: InternalActorRef,
|
||||
receiveTimeout: Option[Duration]): ActorCell =
|
||||
new ActorCell(system, ref, props, supervisor, receiveTimeout) {
|
||||
override def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell =
|
||||
new ActorCell(system, ref, props, supervisor) {
|
||||
override def autoReceiveMessage(msg: Envelope) {
|
||||
msg.message match {
|
||||
case InternalGetActor ⇒ sender ! actor
|
||||
|
|
@ -56,7 +50,17 @@ class TestActorRef[T <: Actor](
|
|||
* thrown will be available to you, while still being able to use
|
||||
* become/unbecome.
|
||||
*/
|
||||
def receive(o: Any): Unit = underlying.receiveMessage(o)
|
||||
def receive(o: Any): Unit = receive(o, underlying.system.deadLetters)
|
||||
|
||||
/**
|
||||
* Directly inject messages into actor receive behavior. Any exceptions
|
||||
* thrown will be available to you, while still being able to use
|
||||
* become/unbecome.
|
||||
*/
|
||||
def receive(o: Any, sender: ActorRef): Unit = try {
|
||||
underlying.currentMessage = Envelope(o, if (sender eq null) underlying.system.deadLetters else sender)(underlying.system)
|
||||
underlying.receiveMessage(o)
|
||||
} finally underlying.currentMessage = null
|
||||
|
||||
/**
|
||||
* Retrieve reference to the underlying actor, where the static type matches the factory used inside the
|
||||
|
|
|
|||
|
|
@ -62,50 +62,28 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test kit for testing actors. Inheriting from this trait enables reception of
|
||||
* replies from actors, which are queued by an internal actor and can be
|
||||
* examined using the `expectMsg...` methods. Assertions and bounds concerning
|
||||
* timing are available in the form of `within` blocks.
|
||||
* Implementation trait behind the [[akka.testkit.TestKit]] class: you may use
|
||||
* this if inheriting from a concrete class is not possible.
|
||||
*
|
||||
* <pre>
|
||||
* class Test extends TestKit(ActorSystem()) {
|
||||
* try {
|
||||
* <b>Use of the trait is discouraged because of potential issues with binary
|
||||
* backwards compatibility in the future, use at own risk.</b>
|
||||
*
|
||||
* val test = system.actorOf(Props[SomeActor]
|
||||
* This trait requires the concrete class mixing it in to provide an
|
||||
* [[akka.actor.ActorSystem]] which is available before this traits’s
|
||||
* constructor is run. The recommended way is this:
|
||||
*
|
||||
* within (1 second) {
|
||||
* test ! SomeWork
|
||||
* expectMsg(Result1) // bounded to 1 second
|
||||
* expectMsg(Result2) // bounded to the remainder of the 1 second
|
||||
* }
|
||||
*
|
||||
* } finally {
|
||||
* system.shutdown()
|
||||
* }
|
||||
* {{{
|
||||
* class MyTest extends TestKitBase {
|
||||
* implicit lazy val system = ActorSystem() // may add arguments here
|
||||
* ...
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* Beware of two points:
|
||||
*
|
||||
* - the ActorSystem passed into the constructor needs to be shutdown,
|
||||
* otherwise thread pools and memory will be leaked
|
||||
* - this trait is not thread-safe (only one actor with one queue, one stack
|
||||
* of `within` blocks); it is expected that the code is executed from a
|
||||
* constructor as shown above, which makes this a non-issue, otherwise take
|
||||
* care not to run tests within a single test class instance in parallel.
|
||||
*
|
||||
* It should be noted that for CI servers and the like all maximum Durations
|
||||
* are scaled using their Duration.dilated method, which uses the
|
||||
* TestKitExtension.Settings.TestTimeFactor settable via akka.conf entry "akka.test.timefactor".
|
||||
*
|
||||
* @author Roland Kuhn
|
||||
* @since 1.1
|
||||
* }}}
|
||||
*/
|
||||
class TestKit(_system: ActorSystem) {
|
||||
trait TestKitBase {
|
||||
|
||||
import TestActor.{ Message, RealMessage, NullMessage }
|
||||
|
||||
implicit val system = _system
|
||||
implicit val system: ActorSystem
|
||||
val testKitSettings = TestKitExtension(system)
|
||||
|
||||
private val queue = new LinkedBlockingDeque[Message]()
|
||||
|
|
@ -579,6 +557,48 @@ class TestKit(_system: ActorSystem) {
|
|||
private def format(u: TimeUnit, d: Duration) = "%.3f %s".format(d.toUnit(u), u.toString.toLowerCase)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test kit for testing actors. Inheriting from this trait enables reception of
|
||||
* replies from actors, which are queued by an internal actor and can be
|
||||
* examined using the `expectMsg...` methods. Assertions and bounds concerning
|
||||
* timing are available in the form of `within` blocks.
|
||||
*
|
||||
* <pre>
|
||||
* class Test extends TestKit(ActorSystem()) {
|
||||
* try {
|
||||
*
|
||||
* val test = system.actorOf(Props[SomeActor]
|
||||
*
|
||||
* within (1 second) {
|
||||
* test ! SomeWork
|
||||
* expectMsg(Result1) // bounded to 1 second
|
||||
* expectMsg(Result2) // bounded to the remainder of the 1 second
|
||||
* }
|
||||
*
|
||||
* } finally {
|
||||
* system.shutdown()
|
||||
* }
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* Beware of two points:
|
||||
*
|
||||
* - the ActorSystem passed into the constructor needs to be shutdown,
|
||||
* otherwise thread pools and memory will be leaked
|
||||
* - this trait is not thread-safe (only one actor with one queue, one stack
|
||||
* of `within` blocks); it is expected that the code is executed from a
|
||||
* constructor as shown above, which makes this a non-issue, otherwise take
|
||||
* care not to run tests within a single test class instance in parallel.
|
||||
*
|
||||
* It should be noted that for CI servers and the like all maximum Durations
|
||||
* are scaled using their Duration.dilated method, which uses the
|
||||
* TestKitExtension.Settings.TestTimeFactor settable via akka.conf entry "akka.test.timefactor".
|
||||
*
|
||||
* @author Roland Kuhn
|
||||
* @since 1.1
|
||||
*/
|
||||
class TestKit(_system: ActorSystem) extends { implicit val system = _system } with TestKitBase
|
||||
|
||||
object TestKit {
|
||||
private[testkit] val testActorId = new AtomicInteger(0)
|
||||
|
||||
|
|
@ -640,22 +660,23 @@ class TestProbe(_application: ActorSystem) extends TestKit(_application) {
|
|||
* Replies will be available for inspection with all of TestKit's assertion
|
||||
* methods.
|
||||
*/
|
||||
def send(actor: ActorRef, msg: AnyRef) = {
|
||||
actor.!(msg)(testActor)
|
||||
}
|
||||
def send(actor: ActorRef, msg: Any): Unit = actor.!(msg)(testActor)
|
||||
|
||||
/**
|
||||
* Forward this message as if in the TestActor's receive method with self.forward.
|
||||
*/
|
||||
def forward(actor: ActorRef, msg: AnyRef = lastMessage.msg) {
|
||||
actor.!(msg)(lastMessage.sender)
|
||||
}
|
||||
def forward(actor: ActorRef, msg: Any = lastMessage.msg): Unit = actor.!(msg)(lastMessage.sender)
|
||||
|
||||
/**
|
||||
* Get sender of last received message.
|
||||
*/
|
||||
def sender = lastMessage.sender
|
||||
|
||||
/**
|
||||
* Send message to the sender of the last dequeued message.
|
||||
*/
|
||||
def reply(msg: Any): Unit = sender.!(msg)(ref)
|
||||
|
||||
}
|
||||
|
||||
object TestProbe {
|
||||
|
|
|
|||
|
|
@ -12,7 +12,6 @@ import akka.util.duration._
|
|||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.CreateChild
|
||||
import akka.actor.DeadLetter
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.dispatch.{ Await, MessageDispatcher }
|
||||
|
|
@ -46,9 +45,13 @@ object AkkaSpec {
|
|||
ConfigFactory.parseMap(map.asJava)
|
||||
}
|
||||
|
||||
def getCallerName: String = {
|
||||
def getCallerName(clazz: Class[_]): String = {
|
||||
val s = Thread.currentThread.getStackTrace map (_.getClassName) drop 1 dropWhile (_ matches ".*AkkaSpec.?$")
|
||||
s.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
|
||||
val reduced = s.lastIndexWhere(_ == clazz.getName) match {
|
||||
case -1 ⇒ s
|
||||
case z ⇒ s drop (z + 1)
|
||||
}
|
||||
reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -56,13 +59,13 @@ object AkkaSpec {
|
|||
abstract class AkkaSpec(_system: ActorSystem)
|
||||
extends TestKit(_system) with WordSpec with MustMatchers with BeforeAndAfterAll {
|
||||
|
||||
def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName, config.withFallback(AkkaSpec.testConf)))
|
||||
def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName(getClass), config.withFallback(AkkaSpec.testConf)))
|
||||
|
||||
def this(s: String) = this(ConfigFactory.parseString(s))
|
||||
|
||||
def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap))
|
||||
|
||||
def this() = this(ActorSystem(AkkaSpec.getCallerName, AkkaSpec.testConf))
|
||||
def this() = this(ActorSystem(AkkaSpec.getCallerName(getClass), AkkaSpec.testConf))
|
||||
|
||||
val log: LoggingAdapter = Logging(system, this.getClass)
|
||||
|
||||
|
|
@ -111,9 +114,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
|
|||
"akka.actor.debug.lifecycle" -> true, "akka.actor.debug.event-stream" -> true,
|
||||
"akka.loglevel" -> "DEBUG", "akka.stdout-loglevel" -> "DEBUG")
|
||||
val system = ActorSystem("AkkaSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(AkkaSpec.testConf))
|
||||
val spec = new AkkaSpec(system) {
|
||||
val ref = Seq(testActor, system.actorOf(Props.empty, "name"))
|
||||
}
|
||||
val spec = new AkkaSpec(system) { val ref = Seq(testActor, system.actorOf(Props.empty, "name")) }
|
||||
spec.ref foreach (_.isTerminated must not be true)
|
||||
system.shutdown()
|
||||
spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds)
|
||||
|
|
|
|||
|
|
@ -246,11 +246,18 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA
|
|||
a.underlying.dispatcher.getClass must be(classOf[Dispatcher])
|
||||
}
|
||||
|
||||
"proxy receive for the underlying actor" in {
|
||||
"proxy receive for the underlying actor without sender" in {
|
||||
val ref = TestActorRef[WorkerActor]
|
||||
ref.receive("work")
|
||||
ref.isTerminated must be(true)
|
||||
}
|
||||
|
||||
"proxy receive for the underlying actor with sender" in {
|
||||
val ref = TestActorRef[WorkerActor]
|
||||
ref.receive("work", testActor)
|
||||
ref.isTerminated must be(true)
|
||||
expectMsg("workDone")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -338,6 +338,11 @@ object AkkaBuild extends Build {
|
|||
// for running only tests by tag use system property: -Dakka.test.tags.only=<tag name>
|
||||
lazy val useOnlyTestTags: Set[String] = systemPropertyAsSeq("akka.test.tags.only").toSet
|
||||
|
||||
def executeMultiJvmTests: Boolean = {
|
||||
useOnlyTestTags.contains("long-running") ||
|
||||
!(useExcludeTestTags -- useIncludeTestTags).contains("long-running")
|
||||
}
|
||||
|
||||
def systemPropertyAsSeq(name: String): Seq[String] = {
|
||||
val prop = System.getProperty(name, "")
|
||||
if (prop.isEmpty) Seq.empty else prop.split(",").toSeq
|
||||
|
|
@ -402,20 +407,22 @@ object AkkaBuild extends Build {
|
|||
|
||||
lazy val multiJvmSettings = MultiJvmPlugin.settings ++ inConfig(MultiJvm)(ScalariformPlugin.scalariformSettings) ++ Seq(
|
||||
compileInputs in MultiJvm <<= (compileInputs in MultiJvm) dependsOn (ScalariformKeys.format in MultiJvm),
|
||||
ScalariformKeys.preferences in MultiJvm := formattingPreferences,
|
||||
if (multiNodeEnabled)
|
||||
executeTests in Test <<= ((executeTests in Test), (multiNodeExecuteTests in MultiJvm)) map {
|
||||
case (tr, mr) =>
|
||||
val r = tr._2 ++ mr._2
|
||||
(Tests.overall(r.values), r)
|
||||
}
|
||||
else
|
||||
executeTests in Test <<= ((executeTests in Test), (executeTests in MultiJvm)) map {
|
||||
case (tr, mr) =>
|
||||
val r = tr._2 ++ mr._2
|
||||
(Tests.overall(r.values), r)
|
||||
}
|
||||
)
|
||||
ScalariformKeys.preferences in MultiJvm := formattingPreferences) ++
|
||||
((executeMultiJvmTests, multiNodeEnabled) match {
|
||||
case (true, true) =>
|
||||
executeTests in Test <<= ((executeTests in Test), (multiNodeExecuteTests in MultiJvm)) map {
|
||||
case ((_, testResults), (_, multiNodeResults)) =>
|
||||
val results = testResults ++ multiNodeResults
|
||||
(Tests.overall(results.values), results)
|
||||
}
|
||||
case (true, false) =>
|
||||
executeTests in Test <<= ((executeTests in Test), (executeTests in MultiJvm)) map {
|
||||
case ((_, testResults), (_, multiNodeResults)) =>
|
||||
val results = testResults ++ multiNodeResults
|
||||
(Tests.overall(results.values), results)
|
||||
}
|
||||
case (false, _) => Seq.empty
|
||||
})
|
||||
|
||||
lazy val mimaSettings = mimaDefaultSettings ++ Seq(
|
||||
// MiMa
|
||||
|
|
@ -477,8 +484,8 @@ object Dependency {
|
|||
|
||||
object V {
|
||||
val Camel = "2.8.0"
|
||||
val Logback = "0.9.28"
|
||||
val Netty = "3.3.0.Final"
|
||||
val Logback = "1.0.4"
|
||||
val Netty = "3.5.0.Final"
|
||||
val Protobuf = "2.4.1"
|
||||
val ScalaStm = "0.5"
|
||||
val Scalatest = "1.6.1"
|
||||
|
|
@ -494,12 +501,6 @@ object Dependency {
|
|||
val slf4jApi = "org.slf4j" % "slf4j-api" % V.Slf4j // MIT
|
||||
val zeroMQ = "org.zeromq" % "zeromq-scala-binding_2.9.1" % "0.0.6" // ApacheV2
|
||||
|
||||
// Runtime
|
||||
|
||||
object Runtime {
|
||||
val logback = "ch.qos.logback" % "logback-classic" % V.Logback % "runtime" // MIT
|
||||
}
|
||||
|
||||
// Test
|
||||
|
||||
object Test {
|
||||
|
|
|
|||
11
project/scripts/multi-node-log-replace
Executable file
11
project/scripts/multi-node-log-replace
Executable file
|
|
@ -0,0 +1,11 @@
|
|||
#!/usr/bin/env bash
|
||||
#
|
||||
# Utility to make log files from multi-node tests easier to analyze.
|
||||
# Replaces jvm names and host:port with corresponding logical role name.
|
||||
#
|
||||
|
||||
|
||||
# check for an sbt command
|
||||
type -P sbt &> /dev/null || fail "sbt command not found"
|
||||
|
||||
sbt "project akka-remote-tests" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2"
|
||||
|
|
@ -93,6 +93,8 @@ fi
|
|||
declare -r version=$1
|
||||
declare -r publish_path="${release_server}:${release_path}"
|
||||
|
||||
[[ `java -version 2>&1 | grep "java version" | awk '{print $3}' | tr -d \" | awk '{split($0, array, ".")} END{print array[2]}'` -eq 6 ]] || fail "Java version is not 1.6"
|
||||
|
||||
# check for a git command
|
||||
type -P git &> /dev/null || fail "git command not found"
|
||||
|
||||
|
|
|
|||
25
scripts/multi-node-log-replace.sh
Executable file
25
scripts/multi-node-log-replace.sh
Executable file
|
|
@ -0,0 +1,25 @@
|
|||
#!/usr/bin/env bash
|
||||
#
|
||||
# Utility to make log files from multi-node tests easier to analyze.
|
||||
# Replaces jvm names and host:port with corresponding logical role name.
|
||||
#
|
||||
# Use with 0, 1 or 2 arguments.
|
||||
#
|
||||
# When using 0 arguments it reads from standard input
|
||||
# and writes to standard output.
|
||||
#
|
||||
# With 1 argument it reads from the file specified in the first argument
|
||||
# and writes to standard output.
|
||||
#
|
||||
# With 2 arguments it reads the file specified in the first argument
|
||||
# and writes to the file specified in the second argument.
|
||||
#
|
||||
# You can also replace the contents of the clipboard instead of using files
|
||||
# by supplying `clipboard` as argument
|
||||
#
|
||||
|
||||
|
||||
# check for an sbt command
|
||||
type -P sbt &> /dev/null || fail "sbt command not found"
|
||||
|
||||
sbt "project akka-remote-tests" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2"
|
||||
Loading…
Add table
Add a link
Reference in a new issue