Merged with master

This commit is contained in:
Jonas Bonér 2012-06-14 16:13:53 +02:00
commit cb0cfac6c7
96 changed files with 2377 additions and 1965 deletions

View file

@ -6,7 +6,7 @@ package akka.actor
import akka.testkit._
import akka.testkit.DefaultTimeout
import akka.testkit.TestEvent._
import akka.dispatch.{ Await, MessageQueueAppendFailedException, BoundedDequeBasedMailbox }
import akka.dispatch.{ Await, BoundedDequeBasedMailbox }
import akka.pattern.ask
import akka.util.duration._
import akka.actor.ActorSystem.Settings
@ -17,16 +17,8 @@ object ActorWithBoundedStashSpec {
class StashingActor(implicit sys: ActorSystem) extends Actor with Stash {
def receive = {
case "hello"
stash()
sender ! "OK"
case "world"
try {
unstashAll()
} catch {
case e: MessageQueueAppendFailedException
expectedException.open()
}
case "hello" stash()
case "world" unstashAll()
}
}
@ -36,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")
}
}
}

View file

@ -36,7 +36,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
"notify with one Terminated message when an Actor is stopped" in {
val terminal = system.actorOf(Props.empty)
startWatching(terminal) ! "hallo"
expectMsg("hallo") // this ensures that the DaemonMsgWatch has been received before we send the PoisonPill
expectMsg("hallo")
terminal ! PoisonPill

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -8,8 +8,8 @@ import java.util.concurrent.atomic.AtomicLong
import akka.dispatch._
import akka.routing._
import akka.AkkaException
import akka.util.{ Switch, Helpers }
import akka.event._
import akka.util.{ NonFatal, Switch, Helpers }
/**
* Interface for all ActorRef providers to implement.
@ -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
}
}

View file

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

View file

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

View file

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

View file

@ -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
// dont 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
// dont ever execute normal message when system message present!
if ((nextMessage eq null) && !isClosed) nextMessage = systemDrain(null)
}
/*
* if we closed the mailbox, we must dump the remaining system messages
* to deadLetters (this is essential for DeathWatch)
*/
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()
}

View file

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

View file

@ -1,19 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.event
import akka.actor._
/**
* The contract of DeathWatch is not properly expressed using the type system
* Whenever there is a publish, all listeners to the Terminated Actor should be atomically removed
* A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down
* See LocalDeathWatch for semantics
*/
abstract class DeathWatch extends ActorEventBus with ActorClassifier {
type Event = Terminated
protected final def classify(event: Event): Classifier = event.actor
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -4,7 +4,6 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
@ -19,58 +18,50 @@ object ClientDowningNodeThatIsUpMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class ClientDowningNodeThatIsUpMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy
class ClientDowningNodeThatIsUpSpec
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy
abstract class ClientDowningNodeThatIsUpSpec
extends MultiNodeSpec(ClientDowningNodeThatIsUpMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender with BeforeAndAfter {
import ClientDowningNodeThatIsUpMultiJvmSpec._
with MultiNodeClusterSpec {
override def initialParticipants = 4
import ClientDowningNodeThatIsUpMultiJvmSpec._
"Client of a 4 node cluster" must {
"be able to DOWN a node that is UP (healthy and available)" taggedAs LongRunningTest in {
val thirdAddress = 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")
}
}
}

View file

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

View file

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

View file

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

View file

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

View file

@ -4,11 +4,10 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.actor.Address
import akka.actor._
import akka.util.duration._
object LeaderDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig {
@ -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")
}
}
}

View file

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

View file

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

View file

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

View file

@ -0,0 +1,61 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import 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")
}
}
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -4,7 +4,6 @@
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
@ -15,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")
}
}
}

View file

@ -0,0 +1,65 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import akka.util.duration._
import scala.collection.immutable.SortedSet
import java.util.concurrent.atomic.AtomicReference
object NodeUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class NodeUpMultiJvmNode1 extends NodeUpSpec with FailureDetectorPuppetStrategy
class NodeUpMultiJvmNode2 extends NodeUpSpec with FailureDetectorPuppetStrategy
abstract class NodeUpSpec
extends MultiNodeSpec(NodeUpMultiJvmSpec)
with MultiNodeClusterSpec {
import NodeUpMultiJvmSpec._
"A cluster node that is joining another cluster" must {
"be moved to UP by the leader after a convergence" taggedAs LongRunningTest in {
awaitClusterUp(first, second)
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")
}
}
}

View file

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

View file

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

View file

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

View file

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

View 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)
}
}
}

View file

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

View file

@ -0,0 +1,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))
}
}
}

View file

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

View file

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

Binary file not shown.

Before

Width:  |  Height:  |  Size: 50 KiB

After

Width:  |  Height:  |  Size: 41 KiB

Before After
Before After

View file

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

View file

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

View file

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

View file

@ -24,4 +24,5 @@ Java API
extending-akka
zeromq
microkernel
testing
howto

View file

@ -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 systems name is appended to a :class:`String` log
Beware that the actor systems 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:

View file

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

View 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.

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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 systems name is appended to a :class:`String` log
Beware that the actor systems 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:

View file

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

View file

@ -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, theres :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
------

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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
}
/**

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -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 traitss
* 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 {

View file

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

View file

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

View file

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

View 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"

View file

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

View 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"