Mergin in the latest master
This commit is contained in:
commit
1711934043
83 changed files with 2846 additions and 1027 deletions
|
|
@ -0,0 +1,17 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.actor;
|
||||
|
||||
public class NonPublicClass {
|
||||
public static Props createProps() {
|
||||
return new Props(MyNonPublicActorClass.class);
|
||||
}
|
||||
}
|
||||
|
||||
class MyNonPublicActorClass extends UntypedActor {
|
||||
@Override public void onReceive(Object msg) {
|
||||
getSender().tell(msg);
|
||||
}
|
||||
}
|
||||
|
|
@ -358,6 +358,13 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
system.stop(serverRef)
|
||||
}
|
||||
|
||||
"support actorOfs where the class of the actor isn't public" in {
|
||||
val a = system.actorOf(NonPublicClass.createProps())
|
||||
a.tell("pigdog", testActor)
|
||||
expectMsg("pigdog")
|
||||
system stop a
|
||||
}
|
||||
|
||||
"stop when sent a poison pill" in {
|
||||
val timeout = Timeout(20000)
|
||||
val ref = system.actorOf(Props(new Actor {
|
||||
|
|
|
|||
|
|
@ -74,6 +74,17 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
|
|||
}
|
||||
}
|
||||
|
||||
"not allow null as subscriber" in {
|
||||
val bus = new EventStream(true)
|
||||
intercept[IllegalArgumentException] { bus.subscribe(null, classOf[M]) }.getMessage must be("subscriber is null")
|
||||
}
|
||||
|
||||
"not allow null as unsubscriber" in {
|
||||
val bus = new EventStream(true)
|
||||
intercept[IllegalArgumentException] { bus.unsubscribe(null, classOf[M]) }.getMessage must be("subscriber is null")
|
||||
intercept[IllegalArgumentException] { bus.unsubscribe(null) }.getMessage must be("subscriber is null")
|
||||
}
|
||||
|
||||
"be able to log unhandled messages" in {
|
||||
val sys = ActorSystem("EventStreamSpecUnhandled", configUnhandled)
|
||||
try {
|
||||
|
|
|
|||
|
|
@ -9,7 +9,6 @@ package akka
|
|||
* <ul>
|
||||
* <li>a uuid for tracking purposes</li>
|
||||
* <li>toString that includes exception name, message and uuid</li>
|
||||
* <li>toLongString which also includes the stack trace</li>
|
||||
* </ul>
|
||||
*/
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
|
|
|
|||
|
|
@ -7,7 +7,6 @@ package akka.actor
|
|||
import akka.AkkaException
|
||||
import scala.reflect.BeanProperty
|
||||
import scala.util.control.NoStackTrace
|
||||
import scala.collection.immutable.Stack
|
||||
import java.util.regex.Pattern
|
||||
|
||||
/**
|
||||
|
|
@ -279,18 +278,14 @@ trait Actor {
|
|||
*/
|
||||
protected[akka] implicit val context: ActorContext = {
|
||||
val contextStack = ActorCell.contextStack.get
|
||||
|
||||
def noContextError =
|
||||
if ((contextStack.isEmpty) || (contextStack.head eq null))
|
||||
throw new ActorInitializationException(
|
||||
"\n\tYou cannot create an instance of [" + getClass.getName + "] explicitly using the constructor (new)." +
|
||||
"\n\tYou have to use one of the factory methods to create a new actor. Either use:" +
|
||||
"\n\t\t'val actor = context.actorOf(Props[MyActor])' (to create a supervised child actor from within an actor), or" +
|
||||
"\n\t\t'val actor = system.actorOf(Props(new MyActor(..)))' (to create a top level actor from the ActorSystem)")
|
||||
|
||||
if (contextStack.isEmpty) noContextError
|
||||
val c = contextStack.head
|
||||
if (c eq null) noContextError
|
||||
ActorCell.contextStack.set(contextStack.push(null))
|
||||
ActorCell.contextStack.set(null :: contextStack)
|
||||
c
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ import akka.japi.Procedure
|
|||
import java.io.{ NotSerializableException, ObjectOutputStream }
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.event.Logging.LogEventException
|
||||
import collection.immutable.{ TreeSet, Stack, TreeMap }
|
||||
import collection.immutable.{ TreeSet, TreeMap }
|
||||
import akka.util.{ Unsafe, Duration, Helpers, NonFatal }
|
||||
|
||||
//TODO: everything here for current compatibility - could be limited more
|
||||
|
|
@ -175,8 +175,8 @@ trait UntypedActorContext extends ActorContext {
|
|||
* for! (waves hand)
|
||||
*/
|
||||
private[akka] object ActorCell {
|
||||
val contextStack = new ThreadLocal[Stack[ActorContext]] {
|
||||
override def initialValue = Stack[ActorContext]()
|
||||
val contextStack = new ThreadLocal[List[ActorContext]] {
|
||||
override def initialValue: List[ActorContext] = Nil
|
||||
}
|
||||
|
||||
final val emptyCancellable: Cancellable = new Cancellable {
|
||||
|
|
@ -186,7 +186,7 @@ private[akka] object ActorCell {
|
|||
|
||||
final val emptyReceiveTimeoutData: (Long, Cancellable) = (-1, emptyCancellable)
|
||||
|
||||
final val behaviorStackPlaceHolder: Stack[Actor.Receive] = Stack.empty.push(Actor.emptyBehavior)
|
||||
final val emptyBehaviorStack: List[Actor.Receive] = Nil
|
||||
|
||||
final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty
|
||||
|
||||
|
|
@ -410,7 +410,7 @@ private[akka] class ActorCell(
|
|||
|
||||
var currentMessage: Envelope = _
|
||||
var actor: Actor = _
|
||||
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
|
||||
private var behaviorStack: List[Actor.Receive] = emptyBehaviorStack
|
||||
@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
|
||||
|
|
@ -513,25 +513,21 @@ private[akka] class ActorCell(
|
|||
|
||||
//This method is in charge of setting up the contextStack and create a new instance of the Actor
|
||||
protected def newActor(): Actor = {
|
||||
contextStack.set(contextStack.get.push(this))
|
||||
contextStack.set(this :: contextStack.get)
|
||||
try {
|
||||
import ActorCell.behaviorStackPlaceHolder
|
||||
|
||||
behaviorStack = behaviorStackPlaceHolder
|
||||
behaviorStack = emptyBehaviorStack
|
||||
val instance = props.creator.apply()
|
||||
|
||||
if (instance eq null)
|
||||
throw new ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'")
|
||||
|
||||
behaviorStack = behaviorStack match {
|
||||
case `behaviorStackPlaceHolder` ⇒ Stack.empty.push(instance.receive)
|
||||
case newBehaviors ⇒ Stack.empty.push(instance.receive).pushAll(newBehaviors.reverse.drop(1))
|
||||
}
|
||||
// If no becomes were issued, the actors behavior is its receive method
|
||||
behaviorStack = if (behaviorStack.isEmpty) instance.receive :: behaviorStack else behaviorStack
|
||||
instance
|
||||
} finally {
|
||||
val stackAfter = contextStack.get
|
||||
if (stackAfter.nonEmpty)
|
||||
contextStack.set(if (stackAfter.head eq null) stackAfter.pop.pop else stackAfter.pop) // pop null marker plus our context
|
||||
contextStack.set(if (stackAfter.head eq null) stackAfter.tail.tail else stackAfter.tail) // pop null marker plus our context
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -686,10 +682,8 @@ private[akka] class ActorCell(
|
|||
}
|
||||
}
|
||||
|
||||
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
|
||||
if (discardOld) unbecome()
|
||||
behaviorStack = behaviorStack.push(behavior)
|
||||
}
|
||||
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit =
|
||||
behaviorStack = behavior :: (if (discardOld && behaviorStack.nonEmpty) behaviorStack.tail else behaviorStack)
|
||||
|
||||
/**
|
||||
* UntypedActorContext impl
|
||||
|
|
@ -704,8 +698,9 @@ private[akka] class ActorCell(
|
|||
|
||||
def unbecome(): Unit = {
|
||||
val original = behaviorStack
|
||||
val popped = original.pop
|
||||
behaviorStack = if (popped.isEmpty) original else popped
|
||||
behaviorStack =
|
||||
if (original.isEmpty || original.tail.isEmpty) actor.receive :: emptyBehaviorStack
|
||||
else original.tail
|
||||
}
|
||||
|
||||
def autoReceiveMessage(msg: Envelope): Unit = {
|
||||
|
|
@ -764,7 +759,7 @@ private[akka] class ActorCell(
|
|||
if (system.settings.DebugLifecycle)
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(a), "stopped"))
|
||||
} finally {
|
||||
behaviorStack = behaviorStackPlaceHolder
|
||||
behaviorStack = emptyBehaviorStack
|
||||
clearActorFields(a)
|
||||
actor = null
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,7 +13,6 @@ import java.io.Closeable
|
|||
import akka.dispatch.Await.{ Awaitable, CanAwait }
|
||||
import akka.util._
|
||||
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
|
||||
import collection.immutable.Stack
|
||||
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
|
||||
|
|
@ -430,7 +429,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
if (!name.matches("""^[a-zA-Z0-9][a-zA-Z0-9-]*$"""))
|
||||
throw new IllegalArgumentException(
|
||||
"invalid ActorSystem name [" + name +
|
||||
"], must contain only word characters (i.e. [a-zA-Z_0-9] plus non-leading '-')")
|
||||
"], must contain only word characters (i.e. [a-zA-Z0-9] plus non-leading '-')")
|
||||
|
||||
import ActorSystem._
|
||||
|
||||
|
|
@ -685,8 +684,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
|
||||
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
|
||||
private val lock = new ReentrantGuard
|
||||
private var callbacks: Stack[Runnable] = _ //non-volatile since guarded by the lock
|
||||
lock withGuard { callbacks = Stack.empty[Runnable] }
|
||||
private var callbacks: List[Runnable] = _ //non-volatile since guarded by the lock
|
||||
lock withGuard { callbacks = Nil }
|
||||
|
||||
private val latch = new CountDownLatch(1)
|
||||
|
||||
|
|
@ -695,17 +694,17 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
|||
case 0 ⇒ throw new RejectedExecutionException("Must be called prior to system shutdown.")
|
||||
case _ ⇒ lock withGuard {
|
||||
if (latch.getCount == 0) throw new RejectedExecutionException("Must be called prior to system shutdown.")
|
||||
else callbacks = callbacks.push(callback)
|
||||
else callbacks ::= callback
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final def run(): Unit = lock withGuard {
|
||||
@tailrec def runNext(c: Stack[Runnable]): Stack[Runnable] = c.headOption match {
|
||||
case None ⇒ Stack.empty[Runnable]
|
||||
case Some(callback) ⇒
|
||||
try callback.run() catch { case e ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) }
|
||||
runNext(c.pop)
|
||||
@tailrec def runNext(c: List[Runnable]): List[Runnable] = c match {
|
||||
case Nil ⇒ Nil
|
||||
case callback :: rest ⇒
|
||||
try callback.run() catch { case NonFatal(e) ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) }
|
||||
runNext(rest)
|
||||
}
|
||||
try { callbacks = runNext(callbacks) } finally latch.countDown()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -188,5 +188,10 @@ case class Props(
|
|||
* able to optimize serialization.
|
||||
*/
|
||||
private[akka] case class FromClassCreator(clazz: Class[_ <: Actor]) extends Function0[Actor] {
|
||||
def apply(): Actor = clazz.newInstance
|
||||
def apply(): Actor = try clazz.newInstance catch {
|
||||
case iae: IllegalAccessException ⇒
|
||||
val ctor = clazz.getDeclaredConstructor()
|
||||
ctor.setAccessible(true)
|
||||
ctor.newInstance()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -324,7 +324,17 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
|||
case some ⇒ some foreach { _ ! event }
|
||||
}
|
||||
|
||||
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber)
|
||||
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = dissociate(from, subscriber)
|
||||
def unsubscribe(subscriber: Subscriber): Unit = dissociate(subscriber)
|
||||
def subscribe(subscriber: Subscriber, to: Classifier): Boolean =
|
||||
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
|
||||
else if (to eq null) throw new IllegalArgumentException("Classifier is null")
|
||||
else associate(to, subscriber)
|
||||
|
||||
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean =
|
||||
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
|
||||
else if (from eq null) throw new IllegalArgumentException("Classifier is null")
|
||||
else dissociate(from, subscriber)
|
||||
|
||||
def unsubscribe(subscriber: Subscriber): Unit =
|
||||
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
|
||||
else dissociate(subscriber)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -41,17 +41,20 @@ class EventStream(private val debug: Boolean = false) extends LoggingBus with Su
|
|||
}
|
||||
|
||||
override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
|
||||
if (subscriber eq null) throw new IllegalArgumentException("subscriber is null")
|
||||
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "subscribing " + subscriber + " to channel " + channel))
|
||||
super.subscribe(subscriber, channel)
|
||||
}
|
||||
|
||||
override def unsubscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
|
||||
if (subscriber eq null) throw new IllegalArgumentException("subscriber is null")
|
||||
val ret = super.unsubscribe(subscriber, channel)
|
||||
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from channel " + channel))
|
||||
ret
|
||||
}
|
||||
|
||||
override def unsubscribe(subscriber: ActorRef) {
|
||||
if (subscriber eq null) throw new IllegalArgumentException("subscriber is null")
|
||||
super.unsubscribe(subscriber)
|
||||
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from all channels"))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -881,3 +881,16 @@ class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class
|
|||
protected def notifyInfo(message: String): Unit = bus.publish(Info(logSource, logClass, message))
|
||||
protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message))
|
||||
}
|
||||
|
||||
private[akka] object NoLogging extends LoggingAdapter {
|
||||
def isErrorEnabled = false
|
||||
def isWarningEnabled = false
|
||||
def isInfoEnabled = false
|
||||
def isDebugEnabled = false
|
||||
|
||||
protected def notifyError(message: String): Unit = ()
|
||||
protected def notifyError(cause: Throwable, message: String): Unit = ()
|
||||
protected def notifyWarning(message: String): Unit = ()
|
||||
protected def notifyInfo(message: String): Unit = ()
|
||||
protected def notifyDebug(message: String): Unit = ()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,16 +11,22 @@ import TimeUnit._
|
|||
import java.lang.{ Double ⇒ JDouble }
|
||||
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
case class Deadline private (time: Duration) {
|
||||
case class Deadline private (time: Duration) extends Ordered[Deadline] {
|
||||
def +(other: Duration): Deadline = copy(time = time + other)
|
||||
def -(other: Duration): Deadline = copy(time = time - other)
|
||||
def -(other: Deadline): Duration = time - other.time
|
||||
def timeLeft: Duration = this - Deadline.now
|
||||
def hasTimeLeft(): Boolean = !isOverdue() //Code reuse FTW
|
||||
def isOverdue(): Boolean = (time.toNanos - System.nanoTime()) < 0
|
||||
def compare(that: Deadline) = this.time compare that.time
|
||||
}
|
||||
|
||||
object Deadline {
|
||||
def now: Deadline = Deadline(Duration(System.nanoTime, NANOSECONDS))
|
||||
|
||||
implicit object DeadlineIsOrdered extends Ordering[Deadline] {
|
||||
def compare(a: Deadline, b: Deadline) = a compare b
|
||||
}
|
||||
}
|
||||
|
||||
object Duration {
|
||||
|
|
|
|||
|
|
@ -165,8 +165,7 @@ class AccrualFailureDetector(
|
|||
else PhiFactor * timestampDiff / mean
|
||||
}
|
||||
|
||||
// FIXME change to debug log level, when failure detector is stable
|
||||
log.info("Phi value [{}] and threshold [{}] for connection [{}] ", phi, threshold, connection)
|
||||
log.debug("Phi value [{}] and threshold [{}] for connection [{}] ", phi, threshold, connection)
|
||||
phi
|
||||
}
|
||||
|
||||
|
|
|
|||
File diff suppressed because it is too large
Load diff
|
|
@ -19,7 +19,7 @@ class VectorClockException(message: String) extends AkkaException(message)
|
|||
*/
|
||||
trait Versioned[T] {
|
||||
def version: VectorClock
|
||||
def +(node: VectorClock.Node): T
|
||||
def :+(node: VectorClock.Node): T
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -142,7 +142,7 @@ case class VectorClock(
|
|||
/**
|
||||
* Increment the version for the node passed as argument. Returns a new VectorClock.
|
||||
*/
|
||||
def +(node: Node): VectorClock = copy(versions = versions + (node -> Timestamp()))
|
||||
def :+(node: Node): VectorClock = copy(versions = versions + (node -> Timestamp()))
|
||||
|
||||
/**
|
||||
* Returns true if <code>this</code> and <code>that</code> are concurrent else false.
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec
|
|||
"Client of a 4 node cluster" must {
|
||||
|
||||
"be able to DOWN a node that is UNREACHABLE (killed)" taggedAs LongRunningTest in {
|
||||
val thirdAddress = node(third).address
|
||||
val thirdAddress = address(third)
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
runOn(first) {
|
||||
|
|
@ -47,23 +47,23 @@ abstract class ClientDowningNodeThatIsUnreachableSpec
|
|||
|
||||
// mark 'third' node as DOWN
|
||||
cluster.down(thirdAddress)
|
||||
testConductor.enter("down-third-node")
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false)
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
testConductor.enter("down-third-node")
|
||||
enterBarrier("down-third-node")
|
||||
}
|
||||
|
||||
runOn(second, fourth) {
|
||||
testConductor.enter("down-third-node")
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion")
|
||||
enterBarrier("await-completion")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -37,13 +37,13 @@ abstract class ClientDowningNodeThatIsUpSpec
|
|||
"Client of a 4 node cluster" must {
|
||||
|
||||
"be able to DOWN a node that is UP (healthy and available)" taggedAs LongRunningTest in {
|
||||
val thirdAddress = node(third).address
|
||||
val thirdAddress = address(third)
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
runOn(first) {
|
||||
// mark 'third' node as DOWN
|
||||
cluster.down(thirdAddress)
|
||||
testConductor.enter("down-third-node")
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
markNodeAsUnavailable(thirdAddress)
|
||||
|
||||
|
|
@ -52,16 +52,16 @@ abstract class ClientDowningNodeThatIsUpSpec
|
|||
}
|
||||
|
||||
runOn(third) {
|
||||
testConductor.enter("down-third-node")
|
||||
enterBarrier("down-third-node")
|
||||
}
|
||||
|
||||
runOn(second, fourth) {
|
||||
testConductor.enter("down-third-node")
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion")
|
||||
enterBarrier("await-completion")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -39,19 +39,19 @@ abstract class ConvergenceSpec
|
|||
|
||||
"A cluster of 3 members" must {
|
||||
|
||||
"reach initial convergence" taggedAs LongRunningTest ignore {
|
||||
"reach initial convergence" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(fourth) {
|
||||
// doesn't join immediately
|
||||
}
|
||||
|
||||
testConductor.enter("after-1")
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"not reach convergence while any nodes are unreachable" taggedAs LongRunningTest ignore {
|
||||
val thirdAddress = node(third).address
|
||||
testConductor.enter("before-shutdown")
|
||||
"not reach convergence while any nodes are unreachable" taggedAs LongRunningTest in {
|
||||
val thirdAddress = address(third)
|
||||
enterBarrier("before-shutdown")
|
||||
|
||||
runOn(first) {
|
||||
// kill 'third' node
|
||||
|
|
@ -60,15 +60,13 @@ abstract class ConvergenceSpec
|
|||
}
|
||||
|
||||
runOn(first, second) {
|
||||
val firstAddress = node(first).address
|
||||
val secondAddress = node(second).address
|
||||
|
||||
within(28 seconds) {
|
||||
// third becomes unreachable
|
||||
awaitCond(cluster.latestGossip.overview.unreachable.size == 1)
|
||||
awaitCond(cluster.latestGossip.members.size == 2)
|
||||
awaitCond(cluster.latestGossip.members.forall(_.status == MemberStatus.Up))
|
||||
awaitSeenSameState(Seq(firstAddress, secondAddress))
|
||||
awaitSeenSameState(first, second)
|
||||
// still one unreachable
|
||||
cluster.latestGossip.overview.unreachable.size must be(1)
|
||||
cluster.latestGossip.overview.unreachable.head.address must be(thirdAddress)
|
||||
|
|
@ -78,30 +76,26 @@ abstract class ConvergenceSpec
|
|||
}
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest ignore {
|
||||
"not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest in {
|
||||
runOn(fourth) {
|
||||
// try to join
|
||||
cluster.join(node(first).address)
|
||||
cluster.join(first)
|
||||
}
|
||||
|
||||
val firstAddress = node(first).address
|
||||
val secondAddress = node(second).address
|
||||
val fourthAddress = node(fourth).address
|
||||
|
||||
def memberStatus(address: Address): Option[MemberStatus] =
|
||||
cluster.latestGossip.members.collectFirst { case m if m.address == address ⇒ m.status }
|
||||
|
||||
def assertNotMovedUp: Unit = {
|
||||
within(20 seconds) {
|
||||
awaitCond(cluster.latestGossip.members.size == 3)
|
||||
awaitSeenSameState(Seq(firstAddress, secondAddress, fourthAddress))
|
||||
memberStatus(firstAddress) must be(Some(MemberStatus.Up))
|
||||
memberStatus(secondAddress) must be(Some(MemberStatus.Up))
|
||||
awaitSeenSameState(first, second, fourth)
|
||||
memberStatus(first) must be(Some(MemberStatus.Up))
|
||||
memberStatus(second) must be(Some(MemberStatus.Up))
|
||||
// leader is not allowed to move the new node to Up
|
||||
memberStatus(fourthAddress) must be(Some(MemberStatus.Joining))
|
||||
memberStatus(fourth) must be(Some(MemberStatus.Joining))
|
||||
// still no convergence
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
|
@ -116,7 +110,7 @@ abstract class ConvergenceSpec
|
|||
}
|
||||
}
|
||||
|
||||
testConductor.enter("after-3")
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,21 +29,17 @@ abstract class GossipingAccrualFailureDetectorSpec
|
|||
|
||||
import GossipingAccrualFailureDetectorMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A Gossip-driven Failure Detector" must {
|
||||
|
||||
"receive gossip heartbeats so that all member nodes in the cluster are marked 'available'" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
5.seconds.dilated.sleep // let them gossip
|
||||
cluster.failureDetector.isAvailable(firstAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(secondAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(thirdAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(first) must be(true)
|
||||
cluster.failureDetector.isAvailable(second) must be(true)
|
||||
cluster.failureDetector.isAvailable(third) must be(true)
|
||||
|
||||
testConductor.enter("after-1")
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"mark node as 'unavailable' if a node in the cluster is shut down (and its heartbeats stops)" taggedAs LongRunningTest in {
|
||||
|
|
@ -53,13 +49,13 @@ abstract class GossipingAccrualFailureDetectorSpec
|
|||
|
||||
runOn(first, second) {
|
||||
// remaning nodes should detect failure...
|
||||
awaitCond(!cluster.failureDetector.isAvailable(thirdAddress), 10.seconds)
|
||||
awaitCond(!cluster.failureDetector.isAvailable(third), 10.seconds)
|
||||
// other connections still ok
|
||||
cluster.failureDetector.isAvailable(firstAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(secondAddress) must be(true)
|
||||
cluster.failureDetector.isAvailable(first) must be(true)
|
||||
cluster.failureDetector.isAvailable(second) must be(true)
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ object JoinTwoClustersMultiJvmSpec extends MultiNodeConfig {
|
|||
val c1 = role("c1")
|
||||
val c2 = role("c2")
|
||||
|
||||
commonConfig(debugConfig(on = true).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy
|
||||
|
|
@ -33,10 +33,6 @@ abstract class JoinTwoClustersSpec
|
|||
|
||||
import JoinTwoClustersMultiJvmSpec._
|
||||
|
||||
lazy val a1Address = node(a1).address
|
||||
lazy val b1Address = node(b1).address
|
||||
lazy val c1Address = node(c1).address
|
||||
|
||||
"Three different clusters (A, B and C)" must {
|
||||
|
||||
"be able to 'elect' a single leader after joining (A -> B)" taggedAs LongRunningTest in {
|
||||
|
|
@ -44,16 +40,16 @@ abstract class JoinTwoClustersSpec
|
|||
runOn(a1, b1, c1) {
|
||||
startClusterNode()
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
enterBarrier("first-started")
|
||||
|
||||
runOn(a1, a2) {
|
||||
cluster.join(a1Address)
|
||||
cluster.join(a1)
|
||||
}
|
||||
runOn(b1, b2) {
|
||||
cluster.join(b1Address)
|
||||
cluster.join(b1)
|
||||
}
|
||||
runOn(c1, c2) {
|
||||
cluster.join(c1Address)
|
||||
cluster.join(c1)
|
||||
}
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 2)
|
||||
|
|
@ -62,10 +58,10 @@ abstract class JoinTwoClustersSpec
|
|||
assertLeader(b1, b2)
|
||||
assertLeader(c1, c2)
|
||||
|
||||
testConductor.enter("two-members")
|
||||
enterBarrier("two-members")
|
||||
|
||||
runOn(b2) {
|
||||
cluster.join(a1Address)
|
||||
cluster.join(a1)
|
||||
}
|
||||
|
||||
runOn(a1, a2, b1, b2) {
|
||||
|
|
@ -75,20 +71,20 @@ abstract class JoinTwoClustersSpec
|
|||
assertLeader(a1, a2, b1, b2)
|
||||
assertLeader(c1, c2)
|
||||
|
||||
testConductor.enter("four-members")
|
||||
enterBarrier("four-members")
|
||||
}
|
||||
|
||||
"be able to 'elect' a single leader after joining (C -> A + B)" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(b2) {
|
||||
cluster.join(c1Address)
|
||||
cluster.join(c1)
|
||||
}
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 6)
|
||||
|
||||
assertLeader(a1, a2, b1, b2, c1, c2)
|
||||
|
||||
testConductor.enter("six-members")
|
||||
enterBarrier("six-members")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -42,11 +42,11 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec
|
|||
"be able to DOWN a 'last' node that is UNREACHABLE" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
val fourthAddress = node(fourth).address
|
||||
val fourthAddress = address(fourth)
|
||||
runOn(first) {
|
||||
// kill 'fourth' node
|
||||
testConductor.shutdown(fourth, 0)
|
||||
testConductor.enter("down-fourth-node")
|
||||
enterBarrier("down-fourth-node")
|
||||
|
||||
// mark the node as unreachable in the failure detector
|
||||
markNodeAsUnavailable(fourthAddress)
|
||||
|
|
@ -57,26 +57,26 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec
|
|||
}
|
||||
|
||||
runOn(fourth) {
|
||||
testConductor.enter("down-fourth-node")
|
||||
enterBarrier("down-fourth-node")
|
||||
}
|
||||
|
||||
runOn(second, third) {
|
||||
testConductor.enter("down-fourth-node")
|
||||
enterBarrier("down-fourth-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion-1")
|
||||
enterBarrier("await-completion-1")
|
||||
}
|
||||
|
||||
"be able to DOWN a 'middle' node that is UNREACHABLE" taggedAs LongRunningTest in {
|
||||
val secondAddress = node(second).address
|
||||
val secondAddress = address(second)
|
||||
|
||||
testConductor.enter("before-down-second-node")
|
||||
enterBarrier("before-down-second-node")
|
||||
runOn(first) {
|
||||
// kill 'second' node
|
||||
testConductor.shutdown(second, 0)
|
||||
testConductor.enter("down-second-node")
|
||||
enterBarrier("down-second-node")
|
||||
|
||||
// mark the node as unreachable in the failure detector
|
||||
markNodeAsUnavailable(secondAddress)
|
||||
|
|
@ -87,16 +87,16 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec
|
|||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("down-second-node")
|
||||
enterBarrier("down-second-node")
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
testConductor.enter("down-second-node")
|
||||
enterBarrier("down-second-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds)
|
||||
}
|
||||
|
||||
testConductor.enter("await-completion-2")
|
||||
enterBarrier("await-completion-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -50,7 +50,7 @@ abstract class LeaderElectionSpec
|
|||
assertLeaderIn(sortedRoles)
|
||||
}
|
||||
|
||||
testConductor.enter("after")
|
||||
enterBarrier("after")
|
||||
}
|
||||
|
||||
def shutdownLeaderAndVerifyNewLeader(alreadyShutdown: Int): Unit = {
|
||||
|
|
@ -63,34 +63,34 @@ abstract class LeaderElectionSpec
|
|||
myself match {
|
||||
|
||||
case `controller` ⇒
|
||||
val leaderAddress = node(leader).address
|
||||
testConductor.enter("before-shutdown")
|
||||
val leaderAddress = address(leader)
|
||||
enterBarrier("before-shutdown")
|
||||
testConductor.shutdown(leader, 0)
|
||||
testConductor.enter("after-shutdown", "after-down", "completed")
|
||||
enterBarrier("after-shutdown", "after-down", "completed")
|
||||
markNodeAsUnavailable(leaderAddress)
|
||||
|
||||
case `leader` ⇒
|
||||
testConductor.enter("before-shutdown", "after-shutdown")
|
||||
enterBarrier("before-shutdown", "after-shutdown")
|
||||
// this node will be shutdown by the controller and doesn't participate in more barriers
|
||||
|
||||
case `aUser` ⇒
|
||||
val leaderAddress = node(leader).address
|
||||
testConductor.enter("before-shutdown", "after-shutdown")
|
||||
val leaderAddress = address(leader)
|
||||
enterBarrier("before-shutdown", "after-shutdown")
|
||||
// user marks the shutdown leader as DOWN
|
||||
cluster.down(leaderAddress)
|
||||
testConductor.enter("after-down", "completed")
|
||||
enterBarrier("after-down", "completed")
|
||||
markNodeAsUnavailable(leaderAddress)
|
||||
|
||||
case _ if remainingRoles.contains(myself) ⇒
|
||||
// remaining cluster nodes, not shutdown
|
||||
testConductor.enter("before-shutdown", "after-shutdown", "after-down")
|
||||
enterBarrier("before-shutdown", "after-shutdown", "after-down")
|
||||
|
||||
awaitUpConvergence(currentRoles.size - 1)
|
||||
val nextExpectedLeader = remainingRoles.head
|
||||
cluster.isLeader must be(myself == nextExpectedLeader)
|
||||
assertLeaderIn(remainingRoles)
|
||||
|
||||
testConductor.enter("completed")
|
||||
enterBarrier("completed")
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,83 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
||||
object LeaderLeavingMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(
|
||||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state
|
||||
unreachable-nodes-reaper-interval = 30 s
|
||||
}""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
}
|
||||
|
||||
class LeaderLeavingMultiJvmNode1 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderLeavingMultiJvmNode2 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy
|
||||
class LeaderLeavingMultiJvmNode3 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class LeaderLeavingSpec
|
||||
extends MultiNodeSpec(LeaderLeavingMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import LeaderLeavingMultiJvmSpec._
|
||||
|
||||
"A LEADER that is LEAVING" must {
|
||||
|
||||
"be moved to LEAVING, then to EXITING, then to REMOVED, then be shut down and then a new LEADER should be elected" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
val oldLeaderAddress = cluster.leader
|
||||
|
||||
if (cluster.isLeader) {
|
||||
|
||||
cluster.leave(oldLeaderAddress)
|
||||
enterBarrier("leader-left")
|
||||
|
||||
// verify that a NEW LEADER have taken over
|
||||
awaitCond(!cluster.isLeader)
|
||||
|
||||
// verify that the LEADER is shut down
|
||||
awaitCond(!cluster.isRunning, 30.seconds.dilated)
|
||||
|
||||
// verify that the LEADER is REMOVED
|
||||
awaitCond(cluster.status == MemberStatus.Removed)
|
||||
|
||||
} else {
|
||||
|
||||
enterBarrier("leader-left")
|
||||
|
||||
// verify that the LEADER is LEAVING
|
||||
awaitCond(cluster.latestGossip.members.exists(m ⇒ m.status == MemberStatus.Leaving && m.address == oldLeaderAddress)) // wait on LEAVING
|
||||
|
||||
// verify that the LEADER is EXITING
|
||||
awaitCond(cluster.latestGossip.members.exists(m ⇒ m.status == MemberStatus.Exiting && m.address == oldLeaderAddress)) // wait on EXITING
|
||||
|
||||
// verify that the LEADER is no longer part of the 'members' set
|
||||
awaitCond(cluster.latestGossip.members.forall(_.address != oldLeaderAddress))
|
||||
|
||||
// verify that the LEADER is not part of the 'unreachable' set
|
||||
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != oldLeaderAddress))
|
||||
|
||||
// verify that we have a new LEADER
|
||||
awaitCond(cluster.leader != oldLeaderAddress)
|
||||
}
|
||||
|
||||
enterBarrier("finished")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -21,7 +21,7 @@ object MembershipChangeListenerExitingMultiJvmSpec extends MultiNodeConfig {
|
|||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
leader-actions-interval = 5 s # increase the leader action task interval
|
||||
unreachable-nodes-reaper-interval = 30 s # turn "off" reaping to unreachable node set
|
||||
unreachable-nodes-reaper-interval = 300 s # turn "off" reaping to unreachable node set
|
||||
}
|
||||
""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
|
|
@ -37,37 +37,33 @@ abstract class MembershipChangeListenerExitingSpec
|
|||
|
||||
import MembershipChangeListenerExitingMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is EXITING" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
testConductor.enter("registered-listener")
|
||||
cluster.leave(secondAddress)
|
||||
enterBarrier("registered-listener")
|
||||
cluster.leave(second)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("registered-listener")
|
||||
enterBarrier("registered-listener")
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
val exitingLatch = TestLatch()
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 3 && members.exists(m ⇒ m.address == secondAddress && m.status == MemberStatus.Exiting))
|
||||
if (members.size == 3 && members.exists(m ⇒ m.address == address(second) && m.status == MemberStatus.Exiting))
|
||||
exitingLatch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("registered-listener")
|
||||
enterBarrier("registered-listener")
|
||||
exitingLatch.await
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
enterBarrier("finished")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -30,32 +30,31 @@ abstract class MembershipChangeListenerJoinSpec
|
|||
|
||||
import MembershipChangeListenerJoinMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is JOINING" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
val joinLatch = TestLatch()
|
||||
val expectedAddresses = Set(first, second) map address
|
||||
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
|
||||
if (members.map(_.address) == expectedAddresses && members.exists(_.status == MemberStatus.Joining))
|
||||
joinLatch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("registered-listener")
|
||||
enterBarrier("registered-listener")
|
||||
|
||||
joinLatch.await
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("registered-listener")
|
||||
cluster.join(firstAddress)
|
||||
enterBarrier("registered-listener")
|
||||
cluster.join(first)
|
||||
}
|
||||
|
||||
testConductor.enter("after")
|
||||
awaitUpConvergence(2)
|
||||
|
||||
enterBarrier("after")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ import com.typesafe.config.ConfigFactory
|
|||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.actor.Address
|
||||
|
||||
object MembershipChangeListenerLeavingMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -19,7 +20,7 @@ object MembershipChangeListenerLeavingMultiJvmSpec extends MultiNodeConfig {
|
|||
debugConfig(on = false)
|
||||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster.leader-actions-interval = 5 s
|
||||
akka.cluster.unreachable-nodes-reaper-interval = 30 s
|
||||
akka.cluster.unreachable-nodes-reaper-interval = 300 s # turn "off"
|
||||
"""))
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
|
@ -34,37 +35,35 @@ abstract class MembershipChangeListenerLeavingSpec
|
|||
|
||||
import MembershipChangeListenerLeavingMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A registered MembershipChangeListener" must {
|
||||
"be notified when new node is LEAVING" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
testConductor.enter("registered-listener")
|
||||
cluster.leave(secondAddress)
|
||||
enterBarrier("registered-listener")
|
||||
cluster.leave(second)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
testConductor.enter("registered-listener")
|
||||
enterBarrier("registered-listener")
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
val latch = TestLatch()
|
||||
val expectedAddresses = Set(first, second, third) map address
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 3 && members.exists(m ⇒ m.address == secondAddress && m.status == MemberStatus.Leaving))
|
||||
if (members.map(_.address) == expectedAddresses &&
|
||||
members.exists(m ⇒ m.address == address(second) && m.status == MemberStatus.Leaving))
|
||||
latch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("registered-listener")
|
||||
enterBarrier("registered-listener")
|
||||
latch.await
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
enterBarrier("finished")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -27,9 +27,6 @@ abstract class MembershipChangeListenerUpSpec
|
|||
|
||||
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 {
|
||||
|
|
@ -38,42 +35,44 @@ abstract class MembershipChangeListenerUpSpec
|
|||
|
||||
runOn(first, second) {
|
||||
val latch = TestLatch()
|
||||
val expectedAddresses = Set(first, second) map address
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 2 && members.forall(_.status == MemberStatus.Up))
|
||||
if (members.map(_.address) == expectedAddresses && members.forall(_.status == MemberStatus.Up))
|
||||
latch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("listener-1-registered")
|
||||
cluster.join(firstAddress)
|
||||
enterBarrier("listener-1-registered")
|
||||
cluster.join(first)
|
||||
latch.await
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
testConductor.enter("listener-1-registered")
|
||||
enterBarrier("listener-1-registered")
|
||||
}
|
||||
|
||||
testConductor.enter("after-1")
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"(when three nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in {
|
||||
|
||||
val latch = TestLatch()
|
||||
val expectedAddresses = Set(first, second, third) map address
|
||||
cluster.registerListener(new MembershipChangeListener {
|
||||
def notify(members: SortedSet[Member]) {
|
||||
if (members.size == 3 && members.forall(_.status == MemberStatus.Up))
|
||||
if (members.map(_.address) == expectedAddresses && members.forall(_.status == MemberStatus.Up))
|
||||
latch.countDown()
|
||||
}
|
||||
})
|
||||
testConductor.enter("listener-2-registered")
|
||||
enterBarrier("listener-2-registered")
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(firstAddress)
|
||||
cluster.join(first)
|
||||
}
|
||||
|
||||
latch.await
|
||||
|
||||
testConductor.enter("after-2")
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,6 +11,11 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
import akka.util.Duration
|
||||
import org.scalatest.Suite
|
||||
import org.scalatest.TestFailedException
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.actor.ActorPath
|
||||
import akka.actor.RootActorPath
|
||||
|
||||
object MultiNodeClusterSpec {
|
||||
def clusterConfig: Config = ConfigFactory.parseString("""
|
||||
|
|
@ -29,10 +34,49 @@ object MultiNodeClusterSpec {
|
|||
""")
|
||||
}
|
||||
|
||||
trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec ⇒
|
||||
trait MultiNodeClusterSpec extends FailureDetectorStrategy with Suite { self: MultiNodeSpec ⇒
|
||||
|
||||
override def initialParticipants = roles.size
|
||||
|
||||
private val cachedAddresses = new ConcurrentHashMap[RoleName, Address]
|
||||
|
||||
/**
|
||||
* Lookup the Address for the role.
|
||||
*
|
||||
* Implicit conversion from RoleName to Address.
|
||||
*
|
||||
* It is cached, which has the implication that stopping
|
||||
* and then restarting a role (jvm) with another address is not
|
||||
* supported.
|
||||
*/
|
||||
implicit def address(role: RoleName): Address = {
|
||||
cachedAddresses.get(role) match {
|
||||
case null ⇒
|
||||
val address = node(role).address
|
||||
cachedAddresses.put(role, address)
|
||||
address
|
||||
case address ⇒ address
|
||||
}
|
||||
}
|
||||
|
||||
// Cluster tests are written so that if previous step (test method) failed
|
||||
// it will most likely not be possible to run next step. This ensures
|
||||
// fail fast of steps after the first failure.
|
||||
private var failed = false
|
||||
override protected def withFixture(test: NoArgTest): Unit = try {
|
||||
if (failed) {
|
||||
val e = new TestFailedException("Previous step failed", 0)
|
||||
// short stack trace
|
||||
e.setStackTrace(e.getStackTrace.take(1))
|
||||
throw e
|
||||
}
|
||||
super.withFixture(test)
|
||||
} catch {
|
||||
case t ⇒
|
||||
failed = true
|
||||
throw t
|
||||
}
|
||||
|
||||
/**
|
||||
* The cluster node instance. Needs to be lazily created.
|
||||
*/
|
||||
|
|
@ -71,14 +115,14 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
|
|||
// make sure that the node-to-join is started before other join
|
||||
startClusterNode()
|
||||
}
|
||||
testConductor.enter(roles.head.name + "-started")
|
||||
enterBarrier(roles.head.name + "-started")
|
||||
if (roles.tail.contains(myself)) {
|
||||
cluster.join(node(roles.head).address)
|
||||
cluster.join(roles.head)
|
||||
}
|
||||
if (upConvergence && roles.contains(myself)) {
|
||||
awaitUpConvergence(numberOfMembers = roles.length)
|
||||
}
|
||||
testConductor.enter(roles.map(_.name).mkString("-") + "-joined")
|
||||
enterBarrier(roles.map(_.name).mkString("-") + "-joined")
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -129,7 +173,7 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
|
|||
/**
|
||||
* Wait until the specified nodes have seen the same gossip overview.
|
||||
*/
|
||||
def awaitSeenSameState(addresses: Seq[Address]): Unit = {
|
||||
def awaitSeenSameState(addresses: Address*): Unit = {
|
||||
awaitCond {
|
||||
val seen = cluster.latestGossip.overview.seen
|
||||
val seenVectorClocks = addresses.flatMap(seen.get(_))
|
||||
|
|
@ -147,10 +191,9 @@ trait MultiNodeClusterSpec extends FailureDetectorStrategy { self: MultiNodeSpec
|
|||
*/
|
||||
implicit val clusterOrdering: Ordering[RoleName] = new Ordering[RoleName] {
|
||||
import Member.addressOrdering
|
||||
def compare(x: RoleName, y: RoleName) = addressOrdering.compare(node(x).address, node(y).address)
|
||||
def compare(x: RoleName, y: RoleName) = addressOrdering.compare(address(x), address(y))
|
||||
}
|
||||
|
||||
def roleName(address: Address): Option[RoleName] = {
|
||||
testConductor.getNodes.await.find(node(_).address == address)
|
||||
}
|
||||
def roleName(addr: Address): Option[RoleName] = roles.find(address(_) == addr)
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,9 +29,6 @@ abstract class NodeJoinSpec
|
|||
|
||||
import NodeJoinMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
|
||||
"A cluster node" must {
|
||||
"join another cluster and get status JOINING - when sending a 'Join' command" taggedAs LongRunningTest in {
|
||||
|
||||
|
|
@ -40,12 +37,12 @@ abstract class NodeJoinSpec
|
|||
}
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(firstAddress)
|
||||
cluster.join(first)
|
||||
}
|
||||
|
||||
awaitCond(cluster.latestGossip.members.exists { member ⇒ member.address == secondAddress && member.status == MemberStatus.Joining })
|
||||
awaitCond(cluster.latestGossip.members.exists { member ⇒ member.address == address(second) && member.status == MemberStatus.Joining })
|
||||
|
||||
testConductor.enter("after")
|
||||
enterBarrier("after")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,9 +18,9 @@ object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig
|
|||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with AccrualFailureDetectorStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
|
||||
class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class NodeLeavingAndExitingAndBeingRemovedSpec
|
||||
extends MultiNodeSpec(NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec)
|
||||
|
|
@ -28,38 +28,34 @@ abstract class NodeLeavingAndExitingAndBeingRemovedSpec
|
|||
|
||||
import NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
val reaperWaitingTime = 30.seconds.dilated
|
||||
|
||||
"A node that is LEAVING a non-singleton cluster" must {
|
||||
|
||||
// FIXME make it work and remove ignore
|
||||
"be moved to EXITING and then to REMOVED by the reaper" taggedAs LongRunningTest ignore {
|
||||
"eventually set to REMOVED by the reaper, and removed from membership ring and seen table" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
cluster.leave(second)
|
||||
}
|
||||
testConductor.enter("second-left")
|
||||
enterBarrier("second-left")
|
||||
|
||||
runOn(first, third) {
|
||||
// verify that the 'second' node is no longer part of the 'members' set
|
||||
awaitCond(cluster.latestGossip.members.forall(_.address != secondAddress), reaperWaitingTime)
|
||||
awaitCond(cluster.latestGossip.members.forall(_.address != address(second)), reaperWaitingTime)
|
||||
|
||||
// verify that the 'second' node is part of the 'unreachable' set
|
||||
awaitCond(cluster.latestGossip.overview.unreachable.exists(_.status == MemberStatus.Removed), reaperWaitingTime)
|
||||
|
||||
// verify node that got removed is 'second' node
|
||||
val isRemoved = cluster.latestGossip.overview.unreachable.find(_.status == MemberStatus.Removed)
|
||||
isRemoved must be('defined)
|
||||
isRemoved.get.address must be(secondAddress)
|
||||
// verify that the 'second' node is not part of the 'unreachable' set
|
||||
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != address(second)), reaperWaitingTime)
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
runOn(second) {
|
||||
// verify that the second node is shut down and has status REMOVED
|
||||
awaitCond(!cluster.isRunning, reaperWaitingTime)
|
||||
awaitCond(cluster.status == MemberStatus.Removed, reaperWaitingTime)
|
||||
}
|
||||
|
||||
enterBarrier("finished")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig {
|
|||
.withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state
|
||||
unreachable-nodes-reaper-interval = 30 s
|
||||
unreachable-nodes-reaper-interval = 300 s # turn "off"
|
||||
}
|
||||
""")
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig)))
|
||||
|
|
@ -36,21 +36,16 @@ abstract class NodeLeavingAndExitingSpec
|
|||
|
||||
import NodeLeavingAndExitingMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A node that is LEAVING a non-singleton cluster" must {
|
||||
|
||||
// FIXME make it work and remove ignore
|
||||
"be moved to EXITING by the leader" taggedAs LongRunningTest ignore {
|
||||
"be moved to EXITING by the leader" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
cluster.leave(second)
|
||||
}
|
||||
testConductor.enter("second-left")
|
||||
enterBarrier("second-left")
|
||||
|
||||
runOn(first, third) {
|
||||
|
||||
|
|
@ -60,16 +55,16 @@ abstract class NodeLeavingAndExitingSpec
|
|||
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving)) // wait on LEAVING
|
||||
val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving) // verify node that left
|
||||
hasLeft must be('defined)
|
||||
hasLeft.get.address must be(secondAddress)
|
||||
hasLeft.get.address must be(address(second))
|
||||
|
||||
// 2. Verify that 'second' node is set to EXITING
|
||||
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Exiting)) // wait on EXITING
|
||||
val hasExited = cluster.latestGossip.members.find(_.status == MemberStatus.Exiting) // verify node that exited
|
||||
hasExited must be('defined)
|
||||
hasExited.get.address must be(secondAddress)
|
||||
hasExited.get.address must be(address(second))
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
enterBarrier("finished")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -30,31 +30,26 @@ abstract class NodeLeavingSpec
|
|||
|
||||
import NodeLeavingMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A node that is LEAVING a non-singleton cluster" must {
|
||||
|
||||
// FIXME make it work and remove ignore
|
||||
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest ignore {
|
||||
"be marked as LEAVING in the converged membership table" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
cluster.leave(secondAddress)
|
||||
cluster.leave(second)
|
||||
}
|
||||
testConductor.enter("second-left")
|
||||
enterBarrier("second-left")
|
||||
|
||||
runOn(first, third) {
|
||||
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving))
|
||||
|
||||
val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving)
|
||||
hasLeft must be('defined)
|
||||
hasLeft.get.address must be(secondAddress)
|
||||
hasLeft.get.address must be(address(second))
|
||||
}
|
||||
|
||||
testConductor.enter("finished")
|
||||
enterBarrier("finished")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,10 +26,6 @@ abstract class NodeMembershipSpec
|
|||
|
||||
import NodeMembershipMultiJvmSpec._
|
||||
|
||||
lazy val firstAddress = node(first).address
|
||||
lazy val secondAddress = node(second).address
|
||||
lazy val thirdAddress = node(third).address
|
||||
|
||||
"A set of connected cluster systems" must {
|
||||
|
||||
"(when two nodes) start gossiping to each other so that both nodes gets the same gossip info" taggedAs LongRunningTest in {
|
||||
|
|
@ -38,35 +34,35 @@ abstract class NodeMembershipSpec
|
|||
runOn(first) {
|
||||
startClusterNode()
|
||||
}
|
||||
testConductor.enter("first-started")
|
||||
enterBarrier("first-started")
|
||||
|
||||
runOn(first, second) {
|
||||
cluster.join(firstAddress)
|
||||
cluster.join(first)
|
||||
awaitCond(cluster.latestGossip.members.size == 2)
|
||||
assertMembers(cluster.latestGossip.members, firstAddress, secondAddress)
|
||||
assertMembers(cluster.latestGossip.members, first, second)
|
||||
awaitCond {
|
||||
cluster.latestGossip.members.forall(_.status == MemberStatus.Up)
|
||||
}
|
||||
awaitCond(cluster.convergence.isDefined)
|
||||
}
|
||||
|
||||
testConductor.enter("after-1")
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"(when three nodes) start gossiping to each other so that all nodes gets the same gossip info" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(firstAddress)
|
||||
cluster.join(first)
|
||||
}
|
||||
|
||||
awaitCond(cluster.latestGossip.members.size == 3)
|
||||
assertMembers(cluster.latestGossip.members, firstAddress, secondAddress, thirdAddress)
|
||||
assertMembers(cluster.latestGossip.members, first, second, third)
|
||||
awaitCond {
|
||||
cluster.latestGossip.members.forall(_.status == MemberStatus.Up)
|
||||
}
|
||||
awaitCond(cluster.convergence.isDefined)
|
||||
|
||||
testConductor.enter("after-2")
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ abstract class NodeUpSpec
|
|||
|
||||
awaitClusterUp(first, second)
|
||||
|
||||
testConductor.enter("after-1")
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"be unaffected when joining again" taggedAs LongRunningTest in {
|
||||
|
|
@ -45,12 +45,12 @@ abstract class NodeUpSpec
|
|||
unexpected.set(members)
|
||||
}
|
||||
})
|
||||
testConductor.enter("listener-registered")
|
||||
enterBarrier("listener-registered")
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(node(first).address)
|
||||
cluster.join(first)
|
||||
}
|
||||
testConductor.enter("joined-again")
|
||||
enterBarrier("joined-again")
|
||||
|
||||
// let it run for a while to make sure that nothing bad happens
|
||||
for (n ← 1 to 20) {
|
||||
|
|
@ -59,7 +59,7 @@ abstract class NodeUpSpec
|
|||
cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true)
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -43,12 +43,12 @@ abstract class SingletonClusterSpec
|
|||
cluster.isSingletonCluster must be(false)
|
||||
assertLeader(first, second)
|
||||
|
||||
testConductor.enter("after-1")
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"become singleton cluster when one node is shutdown" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val secondAddress = node(second).address
|
||||
val secondAddress = address(second)
|
||||
testConductor.shutdown(second, 0)
|
||||
|
||||
markNodeAsUnavailable(secondAddress)
|
||||
|
|
@ -58,7 +58,7 @@ abstract class SingletonClusterSpec
|
|||
assertLeader(first)
|
||||
}
|
||||
|
||||
testConductor.enter("after-2")
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,18 +21,19 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
gossip-interval = 400 ms
|
||||
nr-of-deputy-nodes = 0
|
||||
# FIXME remove this (use default) when ticket #2239 has been fixed
|
||||
gossip-interval = 400 ms
|
||||
}
|
||||
akka.loglevel = INFO
|
||||
"""))
|
||||
}
|
||||
|
||||
class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec with FailureDetectorPuppetStrategy
|
||||
class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
|
||||
class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
|
||||
class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
|
||||
class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
|
||||
class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class SunnyWeatherSpec
|
||||
extends MultiNodeSpec(SunnyWeatherMultiJvmSpec)
|
||||
|
|
@ -62,7 +63,7 @@ abstract class SunnyWeatherSpec
|
|||
})
|
||||
|
||||
for (n ← 1 to 30) {
|
||||
testConductor.enter("period-" + n)
|
||||
enterBarrier("period-" + n)
|
||||
unexpected.get must be(null)
|
||||
awaitUpConvergence(roles.size)
|
||||
assertLeaderIn(roles)
|
||||
|
|
@ -70,7 +71,7 @@ abstract class SunnyWeatherSpec
|
|||
1.seconds.sleep
|
||||
}
|
||||
|
||||
testConductor.enter("after")
|
||||
enterBarrier("after")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,436 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.actor.Address
|
||||
import akka.remote.testconductor.RoleName
|
||||
import MemberStatus._
|
||||
|
||||
object TransitionMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
val fifth = role("fifth")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString(
|
||||
"akka.cluster.periodic-tasks-initial-delay = 300 s # turn off all periodic tasks")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class TransitionMultiJvmNode1 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode2 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode3 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode4 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode5 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class TransitionSpec
|
||||
extends MultiNodeSpec(TransitionMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import TransitionMultiJvmSpec._
|
||||
|
||||
// sorted in the order used by the cluster
|
||||
def leader(roles: RoleName*) = roles.sorted.head
|
||||
def nonLeader(roles: RoleName*) = roles.toSeq.sorted.tail
|
||||
|
||||
def memberStatus(address: Address): MemberStatus = {
|
||||
val statusOption = (cluster.latestGossip.members ++ cluster.latestGossip.overview.unreachable).collectFirst {
|
||||
case m if m.address == address ⇒ m.status
|
||||
}
|
||||
statusOption must not be (None)
|
||||
statusOption.get
|
||||
}
|
||||
|
||||
def memberAddresses: Set[Address] = cluster.latestGossip.members.map(_.address)
|
||||
|
||||
def members: Set[RoleName] = memberAddresses.flatMap(roleName(_))
|
||||
|
||||
def seenLatestGossip: Set[RoleName] = {
|
||||
val gossip = cluster.latestGossip
|
||||
gossip.overview.seen.collect {
|
||||
case (address, v) if v == gossip.version ⇒ roleName(address)
|
||||
}.flatten.toSet
|
||||
}
|
||||
|
||||
def awaitSeen(addresses: Address*): Unit = awaitCond {
|
||||
(seenLatestGossip map address) == addresses.toSet
|
||||
}
|
||||
|
||||
def awaitMembers(addresses: Address*): Unit = awaitCond {
|
||||
memberAddresses == addresses.toSet
|
||||
}
|
||||
|
||||
def awaitMemberStatus(address: Address, status: MemberStatus): Unit = awaitCond {
|
||||
memberStatus(address) == Up
|
||||
}
|
||||
|
||||
// DSL sugar for `role1 gossipTo role2`
|
||||
implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role)
|
||||
var gossipBarrierCounter = 0
|
||||
class RoleWrapper(fromRole: RoleName) {
|
||||
def gossipTo(toRole: RoleName): Unit = {
|
||||
gossipBarrierCounter += 1
|
||||
runOn(toRole) {
|
||||
val g = cluster.latestGossip
|
||||
enterBarrier("before-gossip-" + gossipBarrierCounter)
|
||||
awaitCond(cluster.latestGossip != g) // received gossip
|
||||
enterBarrier("after-gossip-" + gossipBarrierCounter)
|
||||
}
|
||||
runOn(fromRole) {
|
||||
enterBarrier("before-gossip-" + gossipBarrierCounter)
|
||||
cluster.gossipTo(toRole) // send gossip
|
||||
enterBarrier("after-gossip-" + gossipBarrierCounter)
|
||||
}
|
||||
runOn(roles.filterNot(r ⇒ r == fromRole || r == toRole): _*) {
|
||||
enterBarrier("before-gossip-" + gossipBarrierCounter)
|
||||
enterBarrier("after-gossip-" + gossipBarrierCounter)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"A Cluster" must {
|
||||
|
||||
"start nodes as singleton clusters" taggedAs LongRunningTest in {
|
||||
|
||||
startClusterNode()
|
||||
cluster.isSingletonCluster must be(true)
|
||||
cluster.status must be(Joining)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
cluster.leaderActions()
|
||||
cluster.status must be(Up)
|
||||
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"perform correct transitions when second joining first" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(second) {
|
||||
cluster.join(first)
|
||||
}
|
||||
runOn(first) {
|
||||
awaitMembers(first, second)
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Joining)
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
enterBarrier("second-joined")
|
||||
|
||||
first gossipTo second
|
||||
runOn(second) {
|
||||
members must be(Set(first, second))
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Joining)
|
||||
// we got a conflicting version in second, and therefore not convergence in second
|
||||
seenLatestGossip must be(Set(second))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
second gossipTo first
|
||||
runOn(first) {
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
}
|
||||
|
||||
first gossipTo second
|
||||
runOn(second) {
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
}
|
||||
|
||||
runOn(first, second) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Joining)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
enterBarrier("convergence-joining-2")
|
||||
|
||||
runOn(leader(first, second)) {
|
||||
cluster.leaderActions()
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
}
|
||||
enterBarrier("leader-actions-2")
|
||||
|
||||
leader(first, second) gossipTo nonLeader(first, second).head
|
||||
runOn(nonLeader(first, second).head) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
nonLeader(first, second).head gossipTo leader(first, second)
|
||||
runOn(first, second) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"perform correct transitions when third joins second" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(third) {
|
||||
cluster.join(second)
|
||||
}
|
||||
runOn(second) {
|
||||
awaitMembers(first, second, third)
|
||||
cluster.convergence.isDefined must be(false)
|
||||
memberStatus(third) must be(Joining)
|
||||
seenLatestGossip must be(Set(second))
|
||||
}
|
||||
enterBarrier("third-joined-second")
|
||||
|
||||
second gossipTo first
|
||||
runOn(first) {
|
||||
members must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
memberStatus(third) must be(Joining)
|
||||
}
|
||||
|
||||
first gossipTo third
|
||||
runOn(third) {
|
||||
members must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
memberStatus(third) must be(Joining)
|
||||
// conflicting version
|
||||
seenLatestGossip must be(Set(third))
|
||||
}
|
||||
|
||||
third gossipTo first
|
||||
third gossipTo second
|
||||
runOn(first, second) {
|
||||
seenLatestGossip must be(Set(myself, third))
|
||||
}
|
||||
|
||||
first gossipTo second
|
||||
runOn(second) {
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
runOn(first, third) {
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
second gossipTo first
|
||||
second gossipTo third
|
||||
runOn(first, second, third) {
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Joining)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
enterBarrier("convergence-joining-3")
|
||||
|
||||
runOn(leader(first, second, third)) {
|
||||
cluster.leaderActions()
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Up)
|
||||
}
|
||||
enterBarrier("leader-actions-3")
|
||||
|
||||
// leader gossipTo first non-leader
|
||||
leader(first, second, third) gossipTo nonLeader(first, second, third).head
|
||||
runOn(nonLeader(first, second, third).head) {
|
||||
memberStatus(third) must be(Up)
|
||||
seenLatestGossip must be(Set(leader(first, second, third), myself))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
// first non-leader gossipTo the other non-leader
|
||||
nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head
|
||||
runOn(nonLeader(first, second, third).head) {
|
||||
cluster.gossipTo(nonLeader(first, second, third).tail.head)
|
||||
}
|
||||
runOn(nonLeader(first, second, third).tail.head) {
|
||||
memberStatus(third) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
// and back again
|
||||
nonLeader(first, second, third).tail.head gossipTo nonLeader(first, second, third).head
|
||||
runOn(nonLeader(first, second, third).head) {
|
||||
memberStatus(third) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
// first non-leader gossipTo the leader
|
||||
nonLeader(first, second, third).head gossipTo leader(first, second, third)
|
||||
runOn(first, second, third) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"startup a second separated cluster consisting of nodes fourth and fifth" taggedAs LongRunningTest in {
|
||||
runOn(fourth) {
|
||||
cluster.join(fifth)
|
||||
awaitMembers(fourth, fifth)
|
||||
cluster.gossipTo(fifth)
|
||||
awaitSeen(fourth, fifth)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
runOn(fifth) {
|
||||
awaitMembers(fourth, fifth)
|
||||
cluster.gossipTo(fourth)
|
||||
awaitSeen(fourth, fifth)
|
||||
cluster.gossipTo(fourth)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
enterBarrier("fourth-joined-fifth")
|
||||
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"perform correct transitions when second cluster (node fourth) joins first cluster (node third)" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(fourth) {
|
||||
cluster.join(third)
|
||||
}
|
||||
runOn(third) {
|
||||
awaitMembers(first, second, third, fourth)
|
||||
seenLatestGossip must be(Set(third))
|
||||
}
|
||||
enterBarrier("fourth-joined-third")
|
||||
|
||||
third gossipTo second
|
||||
runOn(second) {
|
||||
seenLatestGossip must be(Set(second, third))
|
||||
}
|
||||
|
||||
second gossipTo fourth
|
||||
runOn(fourth) {
|
||||
members must be(roles.toSet)
|
||||
// merge conflict
|
||||
seenLatestGossip must be(Set(fourth))
|
||||
}
|
||||
|
||||
fourth gossipTo first
|
||||
fourth gossipTo second
|
||||
fourth gossipTo third
|
||||
fourth gossipTo fifth
|
||||
runOn(first, second, third, fifth) {
|
||||
members must be(roles.toSet)
|
||||
seenLatestGossip must be(Set(fourth, myself))
|
||||
}
|
||||
|
||||
first gossipTo fifth
|
||||
runOn(fifth) {
|
||||
seenLatestGossip must be(Set(first, fourth, fifth))
|
||||
}
|
||||
|
||||
fifth gossipTo third
|
||||
runOn(third) {
|
||||
seenLatestGossip must be(Set(first, third, fourth, fifth))
|
||||
}
|
||||
|
||||
third gossipTo second
|
||||
runOn(second) {
|
||||
seenLatestGossip must be(roles.toSet)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
second gossipTo first
|
||||
second gossipTo third
|
||||
second gossipTo fourth
|
||||
third gossipTo fifth
|
||||
|
||||
seenLatestGossip must be(roles.toSet)
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Up)
|
||||
memberStatus(fourth) must be(Joining)
|
||||
memberStatus(fifth) must be(Up)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
|
||||
enterBarrier("convergence-joining-3")
|
||||
|
||||
runOn(leader(roles: _*)) {
|
||||
cluster.leaderActions()
|
||||
memberStatus(fourth) must be(Up)
|
||||
seenLatestGossip must be(Set(myself))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
// spread the word
|
||||
for (x :: y :: Nil ← (roles.sorted ++ roles.sorted.dropRight(1)).toList.sliding(2)) {
|
||||
x gossipTo y
|
||||
}
|
||||
|
||||
enterBarrier("spread-5")
|
||||
|
||||
seenLatestGossip must be(roles.toSet)
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Up)
|
||||
memberStatus(fourth) must be(Up)
|
||||
memberStatus(fifth) must be(Up)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in {
|
||||
runOn(fifth) {
|
||||
markNodeAsUnavailable(second)
|
||||
cluster.reapUnreachableMembers()
|
||||
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
|
||||
seenLatestGossip must be(Set(fifth))
|
||||
}
|
||||
|
||||
testConductor.enter("after-second-unavailble")
|
||||
|
||||
// spread the word
|
||||
val gossipRound = List(fifth, fourth, third, first, third, fourth, fifth)
|
||||
for (x :: y :: Nil ← gossipRound.sliding(2)) {
|
||||
x gossipTo y
|
||||
}
|
||||
|
||||
runOn((roles.filterNot(_ == second)): _*) {
|
||||
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
cluster.down(second)
|
||||
awaitMemberStatus(second, Down)
|
||||
}
|
||||
|
||||
testConductor.enter("after-second-down")
|
||||
|
||||
// spread the word
|
||||
val gossipRound2 = List(third, fourth, fifth, first, third, fourth, fifth)
|
||||
for (x :: y :: Nil ← gossipRound2.sliding(2)) {
|
||||
x gossipTo y
|
||||
}
|
||||
|
||||
runOn((roles.filterNot(_ == second)): _*) {
|
||||
cluster.latestGossip.overview.unreachable must contain(Member(second, Down))
|
||||
memberStatus(second) must be(Down)
|
||||
seenLatestGossip must be(Set(first, third, fourth, fifth))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
enterBarrier("after-6")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -33,7 +33,9 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
|||
|
||||
val deterministicRandom = new AtomicInteger
|
||||
|
||||
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], new FailureDetectorPuppet(system)) {
|
||||
val failureDetector = new FailureDetectorPuppet(system)
|
||||
|
||||
val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) {
|
||||
|
||||
override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = {
|
||||
if (addresses.isEmpty) None
|
||||
|
|
@ -64,16 +66,6 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
|||
else _gossipToDeputyProbablity
|
||||
}
|
||||
|
||||
@volatile
|
||||
var _unavailable: Set[Address] = Set.empty
|
||||
|
||||
override val failureDetector = new FailureDetectorPuppet(system) {
|
||||
override def isAvailable(connection: Address): Boolean = {
|
||||
if (_unavailable.contains(connection)) false
|
||||
else super.isAvailable(connection)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
val selfAddress = cluster.self.address
|
||||
|
|
@ -91,7 +83,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
|||
before {
|
||||
cluster._gossipToUnreachableProbablity = 0.0
|
||||
cluster._gossipToDeputyProbablity = 0.0
|
||||
cluster._unavailable = Set.empty
|
||||
addresses foreach failureDetector.remove
|
||||
deterministicRandom.set(0)
|
||||
}
|
||||
|
||||
|
|
@ -110,8 +102,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
|||
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)
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
"accept a few more joining nodes" in {
|
||||
|
|
@ -189,7 +180,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
|||
|
||||
"gossip to random unreachable node" in {
|
||||
val dead = Set(addresses(1))
|
||||
cluster._unavailable = dead
|
||||
dead foreach failureDetector.markNodeAsUnavailable
|
||||
cluster._gossipToUnreachableProbablity = 1.0 // always
|
||||
|
||||
cluster.reapUnreachableMembers()
|
||||
|
|
@ -207,7 +198,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
|
|||
cluster._gossipToDeputyProbablity = -1.0 // real impl
|
||||
// 0 and 2 still alive
|
||||
val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5))
|
||||
cluster._unavailable = dead
|
||||
dead foreach failureDetector.markNodeAsUnavailable
|
||||
|
||||
cluster.reapUnreachableMembers()
|
||||
cluster.latestGossip.overview.unreachable.map(_.address) must be(dead)
|
||||
|
|
|
|||
|
|
@ -12,10 +12,8 @@ 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)
|
||||
|
|
@ -24,19 +22,83 @@ class GossipSpec extends WordSpec with MustMatchers {
|
|||
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 e1 = Member(Address("akka", "sys", "e", 2552), Joining)
|
||||
val e2 = Member(Address("akka", "sys", "e", 2552), Up)
|
||||
|
||||
val g1 = Gossip(members = SortedSet(a1, b1, c1, d1))
|
||||
val g2 = Gossip(members = SortedSet(a2, b2, c2, d2))
|
||||
"A Gossip" must {
|
||||
|
||||
"merge members by status priority" in {
|
||||
|
||||
val g1 = Gossip(members = SortedSet(a1, c1, e1))
|
||||
val g2 = Gossip(members = SortedSet(a2, c2, e2))
|
||||
|
||||
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))
|
||||
merged1.members must be(SortedSet(a2, c1, e1))
|
||||
merged1.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining))
|
||||
|
||||
val merged2 = g2 merge g1
|
||||
merged2.members must be(SortedSet(a1, b2, c1, d2))
|
||||
merged2.members.toSeq.map(_.status) must be(Seq(Up, Removed, Leaving, Removed))
|
||||
merged2.members must be(SortedSet(a2, c1, e1))
|
||||
merged2.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining))
|
||||
|
||||
}
|
||||
|
||||
"merge unreachable by status priority" in {
|
||||
|
||||
val g1 = Gossip(members = Gossip.emptyMembers, overview = GossipOverview(unreachable = Set(a1, b1, c1, d1)))
|
||||
val g2 = Gossip(members = Gossip.emptyMembers, overview = GossipOverview(unreachable = Set(a2, b2, c2, d2)))
|
||||
|
||||
val merged1 = g1 merge g2
|
||||
merged1.overview.unreachable must be(Set(a2, b2, c1, d2))
|
||||
merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed))
|
||||
|
||||
val merged2 = g2 merge g1
|
||||
merged2.overview.unreachable must be(Set(a2, b2, c1, d2))
|
||||
merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed))
|
||||
|
||||
}
|
||||
|
||||
"merge by excluding unreachable from members" in {
|
||||
val g1 = Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(c1, d1)))
|
||||
val g2 = Gossip(members = SortedSet(a2, c2), overview = GossipOverview(unreachable = Set(b2, d2)))
|
||||
|
||||
val merged1 = g1 merge g2
|
||||
merged1.members must be(SortedSet(a2))
|
||||
merged1.members.toSeq.map(_.status) must be(Seq(Joining))
|
||||
merged1.overview.unreachable must be(Set(b2, c1, d2))
|
||||
merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Removed, Leaving, Removed))
|
||||
|
||||
val merged2 = g2 merge g1
|
||||
merged2.members must be(SortedSet(a2))
|
||||
merged2.members.toSeq.map(_.status) must be(Seq(Joining))
|
||||
merged2.overview.unreachable must be(Set(b2, c1, d2))
|
||||
merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Removed, Leaving, Removed))
|
||||
|
||||
}
|
||||
|
||||
"start with fresh seen table after merge" in {
|
||||
val g1 = Gossip(members = SortedSet(a1, e1)).seen(a1.address).seen(a1.address)
|
||||
val g2 = Gossip(members = SortedSet(a2, e2)).seen(e2.address).seen(e2.address)
|
||||
|
||||
val merged1 = g1 merge g2
|
||||
merged1.overview.seen.isEmpty must be(true)
|
||||
|
||||
val merged2 = g2 merge g1
|
||||
merged2.overview.seen.isEmpty must be(true)
|
||||
|
||||
}
|
||||
|
||||
"not have node in both members and unreachable" in intercept[IllegalArgumentException] {
|
||||
Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(b2)))
|
||||
}
|
||||
|
||||
"not have live members with wrong status" in intercept[IllegalArgumentException] {
|
||||
// b2 is Removed
|
||||
Gossip(members = SortedSet(a2, b2))
|
||||
}
|
||||
|
||||
"not have non cluster members in seen table" in intercept[IllegalArgumentException] {
|
||||
Gossip(members = SortedSet(a1, e1)).seen(a1.address).seen(e1.address).seen(b1.address)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,138 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import akka.actor.{ Address, AddressFromURIString }
|
||||
import java.net.InetSocketAddress
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import org.scalatest.WordSpec
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.util.Random
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class MemberOrderingSpec extends WordSpec with MustMatchers {
|
||||
import Member.ordering
|
||||
import Member.addressOrdering
|
||||
import MemberStatus._
|
||||
|
||||
"An Ordering[Member]" must {
|
||||
|
||||
"order non-exiting members by host:port" in {
|
||||
val members = SortedSet.empty[Member] +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1112"), Up) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1113"), Joining) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1111"), Up)
|
||||
|
||||
val seq = members.toSeq
|
||||
seq.size must equal(3)
|
||||
seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Up))
|
||||
seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Up))
|
||||
seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Joining))
|
||||
}
|
||||
|
||||
"order exiting members by last" in {
|
||||
val members = SortedSet.empty[Member] +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1113"), Up) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1111"), Joining)
|
||||
|
||||
val seq = members.toSeq
|
||||
seq.size must equal(3)
|
||||
seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Joining))
|
||||
seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Up))
|
||||
seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting))
|
||||
}
|
||||
|
||||
"order multiple exiting members by last but internally by host:port" in {
|
||||
val members = SortedSet.empty[Member] +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1113"), Leaving) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1111"), Up) +
|
||||
Member(AddressFromURIString("akka://sys@darkstar:1110"), Exiting)
|
||||
|
||||
val seq = members.toSeq
|
||||
seq.size must equal(4)
|
||||
seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Up))
|
||||
seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Leaving))
|
||||
seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1110"), Exiting))
|
||||
seq(3) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting))
|
||||
}
|
||||
|
||||
"be sorted by address correctly" in {
|
||||
import Member.ordering
|
||||
// sorting should be done on host and port, only
|
||||
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
|
||||
val m2 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
|
||||
val m3 = Member(Address("cluster", "sys2", "host2", 8000), MemberStatus.Up)
|
||||
val m4 = Member(Address("cluster", "sys2", "host2", 9000), MemberStatus.Up)
|
||||
val m5 = Member(Address("cluster", "sys1", "host2", 10000), MemberStatus.Up)
|
||||
|
||||
val expected = IndexedSeq(m1, m2, m3, m4, m5)
|
||||
val shuffled = Random.shuffle(expected)
|
||||
shuffled.sorted must be(expected)
|
||||
(SortedSet.empty[Member] ++ shuffled).toIndexedSeq must be(expected)
|
||||
}
|
||||
|
||||
"have stable equals and hashCode" in {
|
||||
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Joining)
|
||||
val m2 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
|
||||
val m3 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
|
||||
|
||||
m1 must be(m2)
|
||||
m1.hashCode must be(m2.hashCode)
|
||||
|
||||
m3 must not be (m2)
|
||||
m3 must not be (m1)
|
||||
}
|
||||
}
|
||||
|
||||
"An Ordering[Address]" must {
|
||||
|
||||
"order addresses by port" in {
|
||||
val addresses = SortedSet.empty[Address] +
|
||||
AddressFromURIString("akka://sys@darkstar:1112") +
|
||||
AddressFromURIString("akka://sys@darkstar:1113") +
|
||||
AddressFromURIString("akka://sys@darkstar:1110") +
|
||||
AddressFromURIString("akka://sys@darkstar:1111")
|
||||
|
||||
val seq = addresses.toSeq
|
||||
seq.size must equal(4)
|
||||
seq(0) must equal(AddressFromURIString("akka://sys@darkstar:1110"))
|
||||
seq(1) must equal(AddressFromURIString("akka://sys@darkstar:1111"))
|
||||
seq(2) must equal(AddressFromURIString("akka://sys@darkstar:1112"))
|
||||
seq(3) must equal(AddressFromURIString("akka://sys@darkstar:1113"))
|
||||
}
|
||||
|
||||
"order addresses by hostname" in {
|
||||
val addresses = SortedSet.empty[Address] +
|
||||
AddressFromURIString("akka://sys@darkstar2:1110") +
|
||||
AddressFromURIString("akka://sys@darkstar1:1110") +
|
||||
AddressFromURIString("akka://sys@darkstar3:1110") +
|
||||
AddressFromURIString("akka://sys@darkstar0:1110")
|
||||
|
||||
val seq = addresses.toSeq
|
||||
seq.size must equal(4)
|
||||
seq(0) must equal(AddressFromURIString("akka://sys@darkstar0:1110"))
|
||||
seq(1) must equal(AddressFromURIString("akka://sys@darkstar1:1110"))
|
||||
seq(2) must equal(AddressFromURIString("akka://sys@darkstar2:1110"))
|
||||
seq(3) must equal(AddressFromURIString("akka://sys@darkstar3:1110"))
|
||||
}
|
||||
|
||||
"order addresses by hostname and port" in {
|
||||
val addresses = SortedSet.empty[Address] +
|
||||
AddressFromURIString("akka://sys@darkstar2:1110") +
|
||||
AddressFromURIString("akka://sys@darkstar0:1111") +
|
||||
AddressFromURIString("akka://sys@darkstar2:1111") +
|
||||
AddressFromURIString("akka://sys@darkstar0:1110")
|
||||
|
||||
val seq = addresses.toSeq
|
||||
seq.size must equal(4)
|
||||
seq(0) must equal(AddressFromURIString("akka://sys@darkstar0:1110"))
|
||||
seq(1) must equal(AddressFromURIString("akka://sys@darkstar0:1111"))
|
||||
seq(2) must equal(AddressFromURIString("akka://sys@darkstar2:1110"))
|
||||
seq(3) must equal(AddressFromURIString("akka://sys@darkstar2:1111"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,45 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster
|
||||
|
||||
import org.scalatest.WordSpec
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import akka.actor.Address
|
||||
import scala.util.Random
|
||||
import scala.collection.immutable.SortedSet
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class MemberSpec extends WordSpec with MustMatchers {
|
||||
|
||||
"Member" must {
|
||||
|
||||
"be sorted by address correctly" in {
|
||||
import Member.ordering
|
||||
// sorting should be done on host and port, only
|
||||
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
|
||||
val m2 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
|
||||
val m3 = Member(Address("cluster", "sys2", "host2", 8000), MemberStatus.Up)
|
||||
val m4 = Member(Address("cluster", "sys2", "host2", 9000), MemberStatus.Up)
|
||||
val m5 = Member(Address("cluster", "sys1", "host2", 10000), MemberStatus.Up)
|
||||
|
||||
val expected = IndexedSeq(m1, m2, m3, m4, m5)
|
||||
val shuffled = Random.shuffle(expected)
|
||||
shuffled.sorted must be(expected)
|
||||
(SortedSet.empty[Member] ++ shuffled).toIndexedSeq must be(expected)
|
||||
}
|
||||
|
||||
"have stable equals and hashCode" in {
|
||||
val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Joining)
|
||||
val m2 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up)
|
||||
val m3 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up)
|
||||
|
||||
m1 must be(m2)
|
||||
m1.hashCode must be(m2.hashCode)
|
||||
|
||||
m3 must not be (m2)
|
||||
m3 must not be (m1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -27,67 +27,67 @@ class VectorClockSpec extends AkkaSpec {
|
|||
|
||||
"pass misc comparison test 1" in {
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + Node("1")
|
||||
val clock3_1 = clock2_1 + Node("2")
|
||||
val clock4_1 = clock3_1 + Node("1")
|
||||
val clock2_1 = clock1_1 :+ Node("1")
|
||||
val clock3_1 = clock2_1 :+ Node("2")
|
||||
val clock4_1 = clock3_1 :+ Node("1")
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + Node("1")
|
||||
val clock3_2 = clock2_2 + Node("2")
|
||||
val clock4_2 = clock3_2 + Node("1")
|
||||
val clock2_2 = clock1_2 :+ Node("1")
|
||||
val clock3_2 = clock2_2 :+ Node("2")
|
||||
val clock4_2 = clock3_2 :+ Node("1")
|
||||
|
||||
clock4_1 <> clock4_2 must be(false)
|
||||
}
|
||||
|
||||
"pass misc comparison test 2" in {
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + Node("1")
|
||||
val clock3_1 = clock2_1 + Node("2")
|
||||
val clock4_1 = clock3_1 + Node("1")
|
||||
val clock2_1 = clock1_1 :+ Node("1")
|
||||
val clock3_1 = clock2_1 :+ Node("2")
|
||||
val clock4_1 = clock3_1 :+ Node("1")
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + Node("1")
|
||||
val clock3_2 = clock2_2 + Node("2")
|
||||
val clock4_2 = clock3_2 + Node("1")
|
||||
val clock5_2 = clock4_2 + Node("3")
|
||||
val clock2_2 = clock1_2 :+ Node("1")
|
||||
val clock3_2 = clock2_2 :+ Node("2")
|
||||
val clock4_2 = clock3_2 :+ Node("1")
|
||||
val clock5_2 = clock4_2 :+ Node("3")
|
||||
|
||||
clock4_1 < clock5_2 must be(true)
|
||||
}
|
||||
|
||||
"pass misc comparison test 3" in {
|
||||
var clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + Node("1")
|
||||
val clock2_1 = clock1_1 :+ Node("1")
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + Node("2")
|
||||
val clock2_2 = clock1_2 :+ Node("2")
|
||||
|
||||
clock2_1 <> clock2_2 must be(true)
|
||||
}
|
||||
|
||||
"pass misc comparison test 4" in {
|
||||
val clock1_3 = VectorClock()
|
||||
val clock2_3 = clock1_3 + Node("1")
|
||||
val clock3_3 = clock2_3 + Node("2")
|
||||
val clock4_3 = clock3_3 + Node("1")
|
||||
val clock2_3 = clock1_3 :+ Node("1")
|
||||
val clock3_3 = clock2_3 :+ Node("2")
|
||||
val clock4_3 = clock3_3 :+ Node("1")
|
||||
|
||||
val clock1_4 = VectorClock()
|
||||
val clock2_4 = clock1_4 + Node("1")
|
||||
val clock3_4 = clock2_4 + Node("1")
|
||||
val clock4_4 = clock3_4 + Node("3")
|
||||
val clock2_4 = clock1_4 :+ Node("1")
|
||||
val clock3_4 = clock2_4 :+ Node("1")
|
||||
val clock4_4 = clock3_4 :+ Node("3")
|
||||
|
||||
clock4_3 <> clock4_4 must be(true)
|
||||
}
|
||||
|
||||
"pass misc comparison test 5" in {
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + Node("2")
|
||||
val clock3_1 = clock2_1 + Node("2")
|
||||
val clock2_1 = clock1_1 :+ Node("2")
|
||||
val clock3_1 = clock2_1 :+ Node("2")
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + Node("1")
|
||||
val clock3_2 = clock2_2 + Node("2")
|
||||
val clock4_2 = clock3_2 + Node("2")
|
||||
val clock5_2 = clock4_2 + Node("3")
|
||||
val clock2_2 = clock1_2 :+ Node("1")
|
||||
val clock3_2 = clock2_2 :+ Node("2")
|
||||
val clock4_2 = clock3_2 :+ Node("2")
|
||||
val clock5_2 = clock4_2 :+ Node("3")
|
||||
|
||||
clock3_1 < clock5_2 must be(true)
|
||||
clock5_2 > clock3_1 must be(true)
|
||||
|
|
@ -95,12 +95,12 @@ class VectorClockSpec extends AkkaSpec {
|
|||
|
||||
"pass misc comparison test 6" in {
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + Node("1")
|
||||
val clock3_1 = clock2_1 + Node("2")
|
||||
val clock2_1 = clock1_1 :+ Node("1")
|
||||
val clock3_1 = clock2_1 :+ Node("2")
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + Node("1")
|
||||
val clock3_2 = clock2_2 + Node("1")
|
||||
val clock2_2 = clock1_2 :+ Node("1")
|
||||
val clock3_2 = clock2_2 :+ Node("1")
|
||||
|
||||
clock3_1 <> clock3_2 must be(true)
|
||||
clock3_2 <> clock3_1 must be(true)
|
||||
|
|
@ -108,14 +108,14 @@ class VectorClockSpec extends AkkaSpec {
|
|||
|
||||
"pass misc comparison test 7" in {
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + Node("1")
|
||||
val clock3_1 = clock2_1 + Node("2")
|
||||
val clock4_1 = clock3_1 + Node("2")
|
||||
val clock5_1 = clock4_1 + Node("3")
|
||||
val clock2_1 = clock1_1 :+ Node("1")
|
||||
val clock3_1 = clock2_1 :+ Node("2")
|
||||
val clock4_1 = clock3_1 :+ Node("2")
|
||||
val clock5_1 = clock4_1 :+ Node("3")
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + Node("2")
|
||||
val clock3_2 = clock2_2 + Node("2")
|
||||
val clock2_2 = clock1_2 :+ Node("2")
|
||||
val clock3_2 = clock2_2 :+ Node("2")
|
||||
|
||||
clock5_1 <> clock3_2 must be(true)
|
||||
clock3_2 <> clock5_1 must be(true)
|
||||
|
|
@ -127,14 +127,14 @@ class VectorClockSpec extends AkkaSpec {
|
|||
val node3 = Node("3")
|
||||
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + node1
|
||||
val clock3_1 = clock2_1 + node2
|
||||
val clock4_1 = clock3_1 + node2
|
||||
val clock5_1 = clock4_1 + node3
|
||||
val clock2_1 = clock1_1 :+ node1
|
||||
val clock3_1 = clock2_1 :+ node2
|
||||
val clock4_1 = clock3_1 :+ node2
|
||||
val clock5_1 = clock4_1 :+ node3
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + node2
|
||||
val clock3_2 = clock2_2 + node2
|
||||
val clock2_2 = clock1_2 :+ node2
|
||||
val clock3_2 = clock2_2 :+ node2
|
||||
|
||||
val merged1 = clock3_2 merge clock5_1
|
||||
merged1.versions.size must be(3)
|
||||
|
|
@ -164,14 +164,14 @@ class VectorClockSpec extends AkkaSpec {
|
|||
val node4 = Node("4")
|
||||
|
||||
val clock1_1 = VectorClock()
|
||||
val clock2_1 = clock1_1 + node1
|
||||
val clock3_1 = clock2_1 + node2
|
||||
val clock4_1 = clock3_1 + node2
|
||||
val clock5_1 = clock4_1 + node3
|
||||
val clock2_1 = clock1_1 :+ node1
|
||||
val clock3_1 = clock2_1 :+ node2
|
||||
val clock4_1 = clock3_1 :+ node2
|
||||
val clock5_1 = clock4_1 :+ node3
|
||||
|
||||
val clock1_2 = VectorClock()
|
||||
val clock2_2 = clock1_2 + node4
|
||||
val clock3_2 = clock2_2 + node4
|
||||
val clock2_2 = clock1_2 :+ node4
|
||||
val clock3_2 = clock2_2 :+ node4
|
||||
|
||||
val merged1 = clock3_2 merge clock5_1
|
||||
merged1.versions.size must be(4)
|
||||
|
|
@ -204,8 +204,8 @@ class VectorClockSpec extends AkkaSpec {
|
|||
val v1 = VectorClock()
|
||||
val v2 = VectorClock()
|
||||
|
||||
val vv1 = v1 + node1
|
||||
val vv2 = v2 + node2
|
||||
val vv1 = v1 :+ node1
|
||||
val vv2 = v2 :+ node2
|
||||
|
||||
(vv1 > v1) must equal(true)
|
||||
(vv2 > v2) must equal(true)
|
||||
|
|
@ -225,12 +225,12 @@ class VectorClockSpec extends AkkaSpec {
|
|||
val a = VectorClock()
|
||||
val b = VectorClock()
|
||||
|
||||
val a1 = a + node1
|
||||
val b1 = b + node2
|
||||
val a1 = a :+ node1
|
||||
val b1 = b :+ node2
|
||||
|
||||
var a2 = a1 + node1
|
||||
var a2 = a1 :+ node1
|
||||
var c = a2.merge(b1)
|
||||
var c1 = c + node3
|
||||
var c1 = c :+ node3
|
||||
|
||||
(c1 > a2) must equal(true)
|
||||
(c1 > b1) must equal(true)
|
||||
|
|
@ -239,7 +239,7 @@ class VectorClockSpec extends AkkaSpec {
|
|||
|
||||
"An instance of Versioned" must {
|
||||
class TestVersioned(val version: VectorClock = VectorClock()) extends Versioned[TestVersioned] {
|
||||
def +(node: Node): TestVersioned = new TestVersioned(version + node)
|
||||
def :+(node: Node): TestVersioned = new TestVersioned(version :+ node)
|
||||
}
|
||||
|
||||
import Versioned.latestVersionOf
|
||||
|
|
@ -251,67 +251,67 @@ class VectorClockSpec extends AkkaSpec {
|
|||
|
||||
"happen before an identical versioned with a single additional event" in {
|
||||
val versioned1_1 = new TestVersioned()
|
||||
val versioned2_1 = versioned1_1 + Node("1")
|
||||
val versioned3_1 = versioned2_1 + Node("2")
|
||||
val versioned4_1 = versioned3_1 + Node("1")
|
||||
val versioned2_1 = versioned1_1 :+ Node("1")
|
||||
val versioned3_1 = versioned2_1 :+ Node("2")
|
||||
val versioned4_1 = versioned3_1 :+ Node("1")
|
||||
|
||||
val versioned1_2 = new TestVersioned()
|
||||
val versioned2_2 = versioned1_2 + Node("1")
|
||||
val versioned3_2 = versioned2_2 + Node("2")
|
||||
val versioned4_2 = versioned3_2 + Node("1")
|
||||
val versioned5_2 = versioned4_2 + Node("3")
|
||||
val versioned2_2 = versioned1_2 :+ Node("1")
|
||||
val versioned3_2 = versioned2_2 :+ Node("2")
|
||||
val versioned4_2 = versioned3_2 :+ Node("1")
|
||||
val versioned5_2 = versioned4_2 :+ Node("3")
|
||||
|
||||
latestVersionOf[TestVersioned](versioned4_1, versioned5_2) must be(versioned5_2)
|
||||
}
|
||||
|
||||
"pass misc comparison test 1" in {
|
||||
var versioned1_1 = new TestVersioned()
|
||||
val versioned2_1 = versioned1_1 + Node("1")
|
||||
val versioned2_1 = versioned1_1 :+ Node("1")
|
||||
|
||||
val versioned1_2 = new TestVersioned()
|
||||
val versioned2_2 = versioned1_2 + Node("2")
|
||||
val versioned2_2 = versioned1_2 :+ Node("2")
|
||||
|
||||
latestVersionOf[TestVersioned](versioned2_1, versioned2_2) must be(versioned2_2)
|
||||
}
|
||||
|
||||
"pass misc comparison test 2" in {
|
||||
val versioned1_3 = new TestVersioned()
|
||||
val versioned2_3 = versioned1_3 + Node("1")
|
||||
val versioned3_3 = versioned2_3 + Node("2")
|
||||
val versioned4_3 = versioned3_3 + Node("1")
|
||||
val versioned2_3 = versioned1_3 :+ Node("1")
|
||||
val versioned3_3 = versioned2_3 :+ Node("2")
|
||||
val versioned4_3 = versioned3_3 :+ Node("1")
|
||||
|
||||
val versioned1_4 = new TestVersioned()
|
||||
val versioned2_4 = versioned1_4 + Node("1")
|
||||
val versioned3_4 = versioned2_4 + Node("1")
|
||||
val versioned4_4 = versioned3_4 + Node("3")
|
||||
val versioned2_4 = versioned1_4 :+ Node("1")
|
||||
val versioned3_4 = versioned2_4 :+ Node("1")
|
||||
val versioned4_4 = versioned3_4 :+ Node("3")
|
||||
|
||||
latestVersionOf[TestVersioned](versioned4_3, versioned4_4) must be(versioned4_4)
|
||||
}
|
||||
|
||||
"pass misc comparison test 3" in {
|
||||
val versioned1_1 = new TestVersioned()
|
||||
val versioned2_1 = versioned1_1 + Node("2")
|
||||
val versioned3_1 = versioned2_1 + Node("2")
|
||||
val versioned2_1 = versioned1_1 :+ Node("2")
|
||||
val versioned3_1 = versioned2_1 :+ Node("2")
|
||||
|
||||
val versioned1_2 = new TestVersioned()
|
||||
val versioned2_2 = versioned1_2 + Node("1")
|
||||
val versioned3_2 = versioned2_2 + Node("2")
|
||||
val versioned4_2 = versioned3_2 + Node("2")
|
||||
val versioned5_2 = versioned4_2 + Node("3")
|
||||
val versioned2_2 = versioned1_2 :+ Node("1")
|
||||
val versioned3_2 = versioned2_2 :+ Node("2")
|
||||
val versioned4_2 = versioned3_2 :+ Node("2")
|
||||
val versioned5_2 = versioned4_2 :+ Node("3")
|
||||
|
||||
latestVersionOf[TestVersioned](versioned3_1, versioned5_2) must be(versioned5_2)
|
||||
}
|
||||
|
||||
"pass misc comparison test 4" in {
|
||||
val versioned1_1 = new TestVersioned()
|
||||
val versioned2_1 = versioned1_1 + Node("1")
|
||||
val versioned3_1 = versioned2_1 + Node("2")
|
||||
val versioned4_1 = versioned3_1 + Node("2")
|
||||
val versioned5_1 = versioned4_1 + Node("3")
|
||||
val versioned2_1 = versioned1_1 :+ Node("1")
|
||||
val versioned3_1 = versioned2_1 :+ Node("2")
|
||||
val versioned4_1 = versioned3_1 :+ Node("2")
|
||||
val versioned5_1 = versioned4_1 :+ Node("3")
|
||||
|
||||
val versioned1_2 = new TestVersioned()
|
||||
val versioned2_2 = versioned1_2 + Node("2")
|
||||
val versioned3_2 = versioned2_2 + Node("2")
|
||||
val versioned2_2 = versioned1_2 :+ Node("2")
|
||||
val versioned3_2 = versioned2_2 :+ Node("2")
|
||||
|
||||
latestVersionOf[TestVersioned](versioned5_1, versioned3_2) must be(versioned3_2)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,8 +5,7 @@
|
|||
Cluster Specification
|
||||
######################
|
||||
|
||||
.. note:: *This document describes the new clustering coming in Akka Coltrane and
|
||||
is not available in the latest stable release)*
|
||||
.. note:: *This document describes the new clustering coming in Akka Coltrane and is not available in the latest stable release)*
|
||||
|
||||
Intro
|
||||
=====
|
||||
|
|
@ -164,8 +163,8 @@ After gossip convergence a ``leader`` for the cluster can be determined. There i
|
|||
``leader`` election process, the ``leader`` can always be recognised deterministically
|
||||
by any node whenever there is gossip convergence. The ``leader`` is simply the first
|
||||
node in sorted order that is able to take the leadership role, where the only
|
||||
allowed member states for a ``leader`` are ``up`` or ``leaving`` (see below for more
|
||||
information about member states).
|
||||
allowed member states for a ``leader`` are ``up``, ``leaving`` or ``exiting`` (see
|
||||
below for more information about member states).
|
||||
|
||||
The role of the ``leader`` is to shift members in and out of the cluster, changing
|
||||
``joining`` members to the ``up`` state or ``exiting`` members to the
|
||||
|
|
@ -302,10 +301,6 @@ handoff has completed then the node will change to the ``exiting`` state. Once
|
|||
all nodes have seen the exiting state (convergence) the ``leader`` will remove the
|
||||
node from the cluster, marking it as ``removed``.
|
||||
|
||||
A node can also be removed forcefully by moving it directly to the ``removed``
|
||||
state using the ``remove`` action. The cluster will rebalance based on the new
|
||||
cluster membership.
|
||||
|
||||
If a node is unreachable then gossip convergence is not possible and therefore
|
||||
any ``leader`` actions are also not possible (for instance, allowing a node to
|
||||
become a part of the cluster, or changing actor distribution). To be able to
|
||||
|
|
@ -314,11 +309,12 @@ unreachable node is experiencing only transient difficulties then it can be
|
|||
explicitly marked as ``down`` using the ``down`` user action. When this node
|
||||
comes back up and begins gossiping it will automatically go through the joining
|
||||
process again. If the unreachable node will be permanently down then it can be
|
||||
removed from the cluster directly with the ``remove`` user action. The cluster
|
||||
can also *auto-down* a node using the accrual failure detector.
|
||||
removed from the cluster directly by shutting the actor system down or killing it
|
||||
through an external ``SIGKILL`` signal, invocation of ``System.exit(status)`` or
|
||||
similar. The cluster can, through the leader, also *auto-down* a node.
|
||||
|
||||
This means that nodes can join and leave the cluster at any point in time,
|
||||
e.g. provide cluster elasticity.
|
||||
This means that nodes can join and leave the cluster at any point in time, i.e.
|
||||
provide cluster elasticity.
|
||||
|
||||
|
||||
State Diagram for the Member States
|
||||
|
|
@ -339,12 +335,12 @@ Member States
|
|||
- **leaving** / **exiting**
|
||||
states during graceful removal
|
||||
|
||||
- **removed**
|
||||
tombstone state (no longer a member)
|
||||
|
||||
- **down**
|
||||
marked as down/offline/unreachable
|
||||
|
||||
- **removed**
|
||||
tombstone state (no longer a member)
|
||||
|
||||
|
||||
User Actions
|
||||
^^^^^^^^^^^^
|
||||
|
|
@ -359,9 +355,6 @@ User Actions
|
|||
- **down**
|
||||
mark a node as temporarily down
|
||||
|
||||
- **remove**
|
||||
remove a node from the cluster immediately
|
||||
|
||||
|
||||
Leader Actions
|
||||
^^^^^^^^^^^^^^
|
||||
|
|
|
|||
|
|
@ -55,7 +55,7 @@ which is a user-level concern.
|
|||
Ordering is preserved on a per-sender basis
|
||||
-------------------------------------------
|
||||
|
||||
Actor ``A1` sends messages ``M1``, ``M2``, ``M3`` to ``A2``
|
||||
Actor ``A1`` sends messages ``M1``, ``M2``, ``M3`` to ``A2``
|
||||
Actor ``A3`` sends messages ``M4``, ``M5``, ``M6`` to ``A2``
|
||||
|
||||
This means that:
|
||||
|
|
|
|||
29
akka-kernel/src/main/dist/bin/akka-cluster
vendored
29
akka-kernel/src/main/dist/bin/akka-cluster
vendored
|
|
@ -63,20 +63,6 @@ case "$2" in
|
|||
$JMX_CLIENT $HOST akka:type=Cluster leave=$ACTOR_SYSTEM_URL
|
||||
;;
|
||||
|
||||
remove)
|
||||
if [ $# -ne 3 ]; then
|
||||
echo "Usage: $SELF <node-hostname:jmx-port> remove <actor-system-url-to-join>"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
ensureNodeIsRunningAndAvailable
|
||||
shift
|
||||
|
||||
ACTOR_SYSTEM_URL=$2
|
||||
echo "Scheduling $ACTOR_SYSTEM_URL to REMOVE"
|
||||
$JMX_CLIENT $HOST akka:type=Cluster remove=$ACTOR_SYSTEM_URL
|
||||
;;
|
||||
|
||||
down)
|
||||
if [ $# -ne 3 ]; then
|
||||
echo "Usage: $SELF <node-hostname:jmx-port> down <actor-system-url-to-join>"
|
||||
|
|
@ -169,19 +155,32 @@ case "$2" in
|
|||
$JMX_CLIENT $HOST akka:type=Cluster Available
|
||||
;;
|
||||
|
||||
is-running)
|
||||
if [ $# -ne 2 ]; then
|
||||
echo "Usage: $SELF <node-hostname:jmx-port> is-running"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
ensureNodeIsRunningAndAvailable
|
||||
shift
|
||||
|
||||
echo "Checking if member node on $HOST is AVAILABLE"
|
||||
$JMX_CLIENT $HOST akka:type=Cluster Running
|
||||
;;
|
||||
|
||||
*)
|
||||
printf "Usage: bin/$SELF <node-hostname:jmx-port> <command> ...\n"
|
||||
printf "\n"
|
||||
printf "Supported commands are:\n"
|
||||
printf "%26s - %s\n" "join <actor-system-url>" "Sends request a JOIN node with the specified URL"
|
||||
printf "%26s - %s\n" "leave <actor-system-url>" "Sends a request for node with URL to LEAVE the cluster"
|
||||
printf "%26s - %s\n" "remove <actor-system-url>" "Sends a request for node with URL to be instantly REMOVED from the cluster"
|
||||
printf "%26s - %s\n" "down <actor-system-url>" "Sends a request for marking node with URL as DOWN"
|
||||
printf "%26s - %s\n" member-status "Asks the member node for its current status"
|
||||
printf "%26s - %s\n" cluster-status "Asks the cluster for its current status (member ring, unavailable nodes, meta data etc.)"
|
||||
printf "%26s - %s\n" leader "Asks the cluster who the current leader is"
|
||||
printf "%26s - %s\n" is-singleton "Checks if the cluster is a singleton cluster (single node cluster)"
|
||||
printf "%26s - %s\n" is-available "Checks if the member node is available"
|
||||
printf "%26s - %s\n" is-running "Checks if the member node is running"
|
||||
printf "%26s - %s\n" has-convergence "Checks if there is a cluster convergence"
|
||||
printf "Where the <actor-system-url> should be on the format of 'akka://actor-system-name@hostname:port'\n"
|
||||
printf "\n"
|
||||
|
|
|
|||
|
|
@ -8,6 +8,81 @@ public final class TestConductorProtocol {
|
|||
public static void registerAllExtensions(
|
||||
com.google.protobuf.ExtensionRegistry registry) {
|
||||
}
|
||||
public enum BarrierOp
|
||||
implements com.google.protobuf.ProtocolMessageEnum {
|
||||
Enter(0, 1),
|
||||
Fail(1, 2),
|
||||
Succeeded(2, 3),
|
||||
Failed(3, 4),
|
||||
;
|
||||
|
||||
public static final int Enter_VALUE = 1;
|
||||
public static final int Fail_VALUE = 2;
|
||||
public static final int Succeeded_VALUE = 3;
|
||||
public static final int Failed_VALUE = 4;
|
||||
|
||||
|
||||
public final int getNumber() { return value; }
|
||||
|
||||
public static BarrierOp valueOf(int value) {
|
||||
switch (value) {
|
||||
case 1: return Enter;
|
||||
case 2: return Fail;
|
||||
case 3: return Succeeded;
|
||||
case 4: return Failed;
|
||||
default: return null;
|
||||
}
|
||||
}
|
||||
|
||||
public static com.google.protobuf.Internal.EnumLiteMap<BarrierOp>
|
||||
internalGetValueMap() {
|
||||
return internalValueMap;
|
||||
}
|
||||
private static com.google.protobuf.Internal.EnumLiteMap<BarrierOp>
|
||||
internalValueMap =
|
||||
new com.google.protobuf.Internal.EnumLiteMap<BarrierOp>() {
|
||||
public BarrierOp findValueByNumber(int number) {
|
||||
return BarrierOp.valueOf(number);
|
||||
}
|
||||
};
|
||||
|
||||
public final com.google.protobuf.Descriptors.EnumValueDescriptor
|
||||
getValueDescriptor() {
|
||||
return getDescriptor().getValues().get(index);
|
||||
}
|
||||
public final com.google.protobuf.Descriptors.EnumDescriptor
|
||||
getDescriptorForType() {
|
||||
return getDescriptor();
|
||||
}
|
||||
public static final com.google.protobuf.Descriptors.EnumDescriptor
|
||||
getDescriptor() {
|
||||
return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(0);
|
||||
}
|
||||
|
||||
private static final BarrierOp[] VALUES = {
|
||||
Enter, Fail, Succeeded, Failed,
|
||||
};
|
||||
|
||||
public static BarrierOp valueOf(
|
||||
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
|
||||
if (desc.getType() != getDescriptor()) {
|
||||
throw new java.lang.IllegalArgumentException(
|
||||
"EnumValueDescriptor is not for this type.");
|
||||
}
|
||||
return VALUES[desc.getIndex()];
|
||||
}
|
||||
|
||||
private final int index;
|
||||
private final int value;
|
||||
|
||||
private BarrierOp(int index, int value) {
|
||||
this.index = index;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(enum_scope:BarrierOp)
|
||||
}
|
||||
|
||||
public enum FailType
|
||||
implements com.google.protobuf.ProtocolMessageEnum {
|
||||
Throttle(0, 1),
|
||||
|
|
@ -56,7 +131,7 @@ public final class TestConductorProtocol {
|
|||
}
|
||||
public static final com.google.protobuf.Descriptors.EnumDescriptor
|
||||
getDescriptor() {
|
||||
return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(0);
|
||||
return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(1);
|
||||
}
|
||||
|
||||
private static final FailType[] VALUES = {
|
||||
|
|
@ -128,7 +203,7 @@ public final class TestConductorProtocol {
|
|||
}
|
||||
public static final com.google.protobuf.Descriptors.EnumDescriptor
|
||||
getDescriptor() {
|
||||
return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(1);
|
||||
return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(2);
|
||||
}
|
||||
|
||||
private static final Direction[] VALUES = {
|
||||
|
|
@ -1699,9 +1774,13 @@ public final class TestConductorProtocol {
|
|||
boolean hasName();
|
||||
String getName();
|
||||
|
||||
// optional bool status = 2;
|
||||
boolean hasStatus();
|
||||
boolean getStatus();
|
||||
// required .BarrierOp op = 2;
|
||||
boolean hasOp();
|
||||
akka.remote.testconductor.TestConductorProtocol.BarrierOp getOp();
|
||||
|
||||
// optional int64 timeout = 3;
|
||||
boolean hasTimeout();
|
||||
long getTimeout();
|
||||
}
|
||||
public static final class EnterBarrier extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
|
|
@ -1764,19 +1843,30 @@ public final class TestConductorProtocol {
|
|||
}
|
||||
}
|
||||
|
||||
// optional bool status = 2;
|
||||
public static final int STATUS_FIELD_NUMBER = 2;
|
||||
private boolean status_;
|
||||
public boolean hasStatus() {
|
||||
// required .BarrierOp op = 2;
|
||||
public static final int OP_FIELD_NUMBER = 2;
|
||||
private akka.remote.testconductor.TestConductorProtocol.BarrierOp op_;
|
||||
public boolean hasOp() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public boolean getStatus() {
|
||||
return status_;
|
||||
public akka.remote.testconductor.TestConductorProtocol.BarrierOp getOp() {
|
||||
return op_;
|
||||
}
|
||||
|
||||
// optional int64 timeout = 3;
|
||||
public static final int TIMEOUT_FIELD_NUMBER = 3;
|
||||
private long timeout_;
|
||||
public boolean hasTimeout() {
|
||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
||||
}
|
||||
public long getTimeout() {
|
||||
return timeout_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
name_ = "";
|
||||
status_ = false;
|
||||
op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter;
|
||||
timeout_ = 0L;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
|
@ -1787,6 +1877,10 @@ public final class TestConductorProtocol {
|
|||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
if (!hasOp()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
memoizedIsInitialized = 1;
|
||||
return true;
|
||||
}
|
||||
|
|
@ -1798,7 +1892,10 @@ public final class TestConductorProtocol {
|
|||
output.writeBytes(1, getNameBytes());
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeBool(2, status_);
|
||||
output.writeEnum(2, op_.getNumber());
|
||||
}
|
||||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
output.writeInt64(3, timeout_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
|
@ -1815,7 +1912,11 @@ public final class TestConductorProtocol {
|
|||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(2, status_);
|
||||
.computeEnumSize(2, op_.getNumber());
|
||||
}
|
||||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeInt64Size(3, timeout_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
|
|
@ -1943,8 +2044,10 @@ public final class TestConductorProtocol {
|
|||
super.clear();
|
||||
name_ = "";
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
status_ = false;
|
||||
op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter;
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
timeout_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
@ -1990,7 +2093,11 @@ public final class TestConductorProtocol {
|
|||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
result.status_ = status_;
|
||||
result.op_ = op_;
|
||||
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
to_bitField0_ |= 0x00000004;
|
||||
}
|
||||
result.timeout_ = timeout_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
|
@ -2010,8 +2117,11 @@ public final class TestConductorProtocol {
|
|||
if (other.hasName()) {
|
||||
setName(other.getName());
|
||||
}
|
||||
if (other.hasStatus()) {
|
||||
setStatus(other.getStatus());
|
||||
if (other.hasOp()) {
|
||||
setOp(other.getOp());
|
||||
}
|
||||
if (other.hasTimeout()) {
|
||||
setTimeout(other.getTimeout());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
|
|
@ -2022,6 +2132,10 @@ public final class TestConductorProtocol {
|
|||
|
||||
return false;
|
||||
}
|
||||
if (!hasOp()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
@ -2054,8 +2168,19 @@ public final class TestConductorProtocol {
|
|||
break;
|
||||
}
|
||||
case 16: {
|
||||
int rawValue = input.readEnum();
|
||||
akka.remote.testconductor.TestConductorProtocol.BarrierOp value = akka.remote.testconductor.TestConductorProtocol.BarrierOp.valueOf(rawValue);
|
||||
if (value == null) {
|
||||
unknownFields.mergeVarintField(2, rawValue);
|
||||
} else {
|
||||
bitField0_ |= 0x00000002;
|
||||
status_ = input.readBool();
|
||||
op_ = value;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 24: {
|
||||
bitField0_ |= 0x00000004;
|
||||
timeout_ = input.readInt64();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
@ -2100,23 +2225,47 @@ public final class TestConductorProtocol {
|
|||
onChanged();
|
||||
}
|
||||
|
||||
// optional bool status = 2;
|
||||
private boolean status_ ;
|
||||
public boolean hasStatus() {
|
||||
// required .BarrierOp op = 2;
|
||||
private akka.remote.testconductor.TestConductorProtocol.BarrierOp op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter;
|
||||
public boolean hasOp() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public boolean getStatus() {
|
||||
return status_;
|
||||
public akka.remote.testconductor.TestConductorProtocol.BarrierOp getOp() {
|
||||
return op_;
|
||||
}
|
||||
public Builder setOp(akka.remote.testconductor.TestConductorProtocol.BarrierOp value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
public Builder setStatus(boolean value) {
|
||||
bitField0_ |= 0x00000002;
|
||||
status_ = value;
|
||||
op_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearStatus() {
|
||||
public Builder clearOp() {
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
status_ = false;
|
||||
op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional int64 timeout = 3;
|
||||
private long timeout_ ;
|
||||
public boolean hasTimeout() {
|
||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
||||
}
|
||||
public long getTimeout() {
|
||||
return timeout_;
|
||||
}
|
||||
public Builder setTimeout(long value) {
|
||||
bitField0_ |= 0x00000004;
|
||||
timeout_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearTimeout() {
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
timeout_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
|
@ -4056,19 +4205,21 @@ public final class TestConductorProtocol {
|
|||
"\0132\r.EnterBarrier\022\037\n\007failure\030\003 \001(\0132\016.Inje" +
|
||||
"ctFailure\022\014\n\004done\030\004 \001(\t\022\035\n\004addr\030\005 \001(\0132\017." +
|
||||
"AddressRequest\"0\n\005Hello\022\014\n\004name\030\001 \002(\t\022\031\n" +
|
||||
"\007address\030\002 \002(\0132\010.Address\",\n\014EnterBarrier" +
|
||||
"\022\014\n\004name\030\001 \002(\t\022\016\n\006status\030\002 \001(\010\"6\n\016Addres" +
|
||||
"sRequest\022\014\n\004node\030\001 \002(\t\022\026\n\004addr\030\002 \001(\0132\010.A" +
|
||||
"ddress\"G\n\007Address\022\020\n\010protocol\030\001 \002(\t\022\016\n\006s" +
|
||||
"ystem\030\002 \002(\t\022\014\n\004host\030\003 \002(\t\022\014\n\004port\030\004 \002(\005\"",
|
||||
"\212\001\n\rInjectFailure\022\032\n\007failure\030\001 \002(\0162\t.Fai" +
|
||||
"lType\022\035\n\tdirection\030\002 \001(\0162\n.Direction\022\031\n\007" +
|
||||
"address\030\003 \001(\0132\010.Address\022\020\n\010rateMBit\030\006 \001(" +
|
||||
"\002\022\021\n\texitValue\030\007 \001(\005*A\n\010FailType\022\014\n\010Thro" +
|
||||
"ttle\020\001\022\016\n\nDisconnect\020\002\022\t\n\005Abort\020\003\022\014\n\010Shu" +
|
||||
"tdown\020\004*,\n\tDirection\022\010\n\004Send\020\001\022\013\n\007Receiv" +
|
||||
"e\020\002\022\010\n\004Both\020\003B\035\n\031akka.remote.testconduct" +
|
||||
"orH\001"
|
||||
"\007address\030\002 \002(\0132\010.Address\"E\n\014EnterBarrier" +
|
||||
"\022\014\n\004name\030\001 \002(\t\022\026\n\002op\030\002 \002(\0162\n.BarrierOp\022\017" +
|
||||
"\n\007timeout\030\003 \001(\003\"6\n\016AddressRequest\022\014\n\004nod" +
|
||||
"e\030\001 \002(\t\022\026\n\004addr\030\002 \001(\0132\010.Address\"G\n\007Addre" +
|
||||
"ss\022\020\n\010protocol\030\001 \002(\t\022\016\n\006system\030\002 \002(\t\022\014\n\004",
|
||||
"host\030\003 \002(\t\022\014\n\004port\030\004 \002(\005\"\212\001\n\rInjectFailu" +
|
||||
"re\022\032\n\007failure\030\001 \002(\0162\t.FailType\022\035\n\tdirect" +
|
||||
"ion\030\002 \001(\0162\n.Direction\022\031\n\007address\030\003 \001(\0132\010" +
|
||||
".Address\022\020\n\010rateMBit\030\006 \001(\002\022\021\n\texitValue\030" +
|
||||
"\007 \001(\005*;\n\tBarrierOp\022\t\n\005Enter\020\001\022\010\n\004Fail\020\002\022" +
|
||||
"\r\n\tSucceeded\020\003\022\n\n\006Failed\020\004*A\n\010FailType\022\014" +
|
||||
"\n\010Throttle\020\001\022\016\n\nDisconnect\020\002\022\t\n\005Abort\020\003\022" +
|
||||
"\014\n\010Shutdown\020\004*,\n\tDirection\022\010\n\004Send\020\001\022\013\n\007" +
|
||||
"Receive\020\002\022\010\n\004Both\020\003B\035\n\031akka.remote.testc" +
|
||||
"onductorH\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
|
@ -4096,7 +4247,7 @@ public final class TestConductorProtocol {
|
|||
internal_static_EnterBarrier_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_EnterBarrier_descriptor,
|
||||
new java.lang.String[] { "Name", "Status", },
|
||||
new java.lang.String[] { "Name", "Op", "Timeout", },
|
||||
akka.remote.testconductor.TestConductorProtocol.EnterBarrier.class,
|
||||
akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder.class);
|
||||
internal_static_AddressRequest_descriptor =
|
||||
|
|
|
|||
|
|
@ -7,8 +7,10 @@ option optimize_for = SPEED;
|
|||
|
||||
/******************************************
|
||||
Compile with:
|
||||
cd ./akka-remote/src/main/protocol
|
||||
cd ./akka-remote-tests/src/main/protocol
|
||||
protoc TestConductorProtocol.proto --java_out ../java
|
||||
cd ../../../..
|
||||
./scripts/fix-protobuf.sh
|
||||
*******************************************/
|
||||
|
||||
message Wrapper {
|
||||
|
|
@ -24,9 +26,17 @@ message Hello {
|
|||
required Address address = 2;
|
||||
}
|
||||
|
||||
enum BarrierOp {
|
||||
Enter = 1;
|
||||
Fail = 2;
|
||||
Succeeded = 3;
|
||||
Failed = 4;
|
||||
}
|
||||
|
||||
message EnterBarrier {
|
||||
required string name = 1;
|
||||
optional bool status = 2;
|
||||
required BarrierOp op = 2;
|
||||
optional int64 timeout = 3;
|
||||
}
|
||||
|
||||
message AddressRequest {
|
||||
|
|
@ -47,11 +57,13 @@ enum FailType {
|
|||
Abort = 3;
|
||||
Shutdown = 4;
|
||||
}
|
||||
|
||||
enum Direction {
|
||||
Send = 1;
|
||||
Receive = 2;
|
||||
Both = 3;
|
||||
}
|
||||
|
||||
message InjectFailure {
|
||||
required FailType failure = 1;
|
||||
optional Direction direction = 2;
|
||||
|
|
|
|||
|
|
@ -8,8 +8,6 @@ import RemoteConnection.getAddrString
|
|||
import TestConductorProtocol._
|
||||
import org.jboss.netty.channel.{ Channel, SimpleChannelUpstreamHandler, ChannelHandlerContext, ChannelStateEvent, MessageEvent }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.Timeout
|
||||
import akka.util.Duration
|
||||
import akka.util.duration._
|
||||
import akka.pattern.ask
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
|
|
@ -26,6 +24,7 @@ import akka.actor.OneForOneStrategy
|
|||
import akka.actor.SupervisorStrategy
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.actor.Status
|
||||
import akka.util.{ Deadline, Timeout, Duration }
|
||||
|
||||
sealed trait Direction {
|
||||
def includes(other: Direction): Boolean
|
||||
|
|
@ -376,7 +375,8 @@ private[akka] class Controller(private var initialParticipants: Int, controllerP
|
|||
* BarrierTimeouts in the players).
|
||||
*/
|
||||
override def supervisorStrategy = OneForOneStrategy() {
|
||||
case BarrierTimeout(data) ⇒ SupervisorStrategy.Resume
|
||||
case BarrierTimeout(data) ⇒ failBarrier(data)
|
||||
case FailedBarrier(data) ⇒ failBarrier(data)
|
||||
case BarrierEmpty(data, msg) ⇒ SupervisorStrategy.Resume
|
||||
case WrongBarrier(name, client, data) ⇒ client ! ToClient(BarrierResult(name, false)); failBarrier(data)
|
||||
case ClientLost(data, node) ⇒ failBarrier(data)
|
||||
|
|
@ -426,6 +426,7 @@ private[akka] class Controller(private var initialParticipants: Int, controllerP
|
|||
case op: ServerOp ⇒
|
||||
op match {
|
||||
case _: EnterBarrier ⇒ barrier forward op
|
||||
case _: FailBarrier ⇒ barrier forward op
|
||||
case GetAddress(node) ⇒
|
||||
if (nodes contains node) sender ! ToClient(AddressReply(node, nodes(node).addr))
|
||||
else addrInterest += node -> ((addrInterest get node getOrElse Set()) + sender)
|
||||
|
|
@ -463,7 +464,7 @@ private[akka] object BarrierCoordinator {
|
|||
|
||||
case class RemoveClient(name: RoleName)
|
||||
|
||||
case class Data(clients: Set[Controller.NodeInfo], barrier: String, arrived: List[ActorRef])
|
||||
case class Data(clients: Set[Controller.NodeInfo], barrier: String, arrived: List[ActorRef], deadline: Deadline)
|
||||
|
||||
trait Printer { this: Product with Throwable with NoStackTrace ⇒
|
||||
override def toString = productPrefix + productIterator.mkString("(", ", ", ")")
|
||||
|
|
@ -471,6 +472,8 @@ private[akka] object BarrierCoordinator {
|
|||
|
||||
case class BarrierTimeout(data: Data)
|
||||
extends RuntimeException("timeout while waiting for barrier '" + data.barrier + "'") with NoStackTrace with Printer
|
||||
case class FailedBarrier(data: Data)
|
||||
extends RuntimeException("failing barrier '" + data.barrier + "'") with NoStackTrace with Printer
|
||||
case class DuplicateNode(data: Data, node: Controller.NodeInfo)
|
||||
extends RuntimeException(node.toString) with NoStackTrace with Printer
|
||||
case class WrongBarrier(barrier: String, client: ActorRef, data: Data)
|
||||
|
|
@ -497,21 +500,23 @@ private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoor
|
|||
import BarrierCoordinator._
|
||||
import akka.actor.FSM._
|
||||
import Controller._
|
||||
import akka.util.{ Timeout ⇒ auTimeout }
|
||||
|
||||
// this shall be set to false if all subsequent barriers shall fail
|
||||
// this shall be set to true if all subsequent barriers shall fail
|
||||
var failed = false
|
||||
|
||||
override def preRestart(reason: Throwable, message: Option[Any]) {}
|
||||
override def postRestart(reason: Throwable) { failed = true }
|
||||
|
||||
// TODO what happens with the other waiting players in case of a test failure?
|
||||
|
||||
startWith(Idle, Data(Set(), "", Nil))
|
||||
startWith(Idle, Data(Set(), "", Nil, null))
|
||||
|
||||
whenUnhandled {
|
||||
case Event(n: NodeInfo, d @ Data(clients, _, _)) ⇒
|
||||
case Event(n: NodeInfo, d @ Data(clients, _, _, _)) ⇒
|
||||
if (clients.find(_.name == n.name).isDefined) throw new DuplicateNode(d, n)
|
||||
stay using d.copy(clients = clients + n)
|
||||
case Event(ClientDisconnected(name), d @ Data(clients, _, arrived)) ⇒
|
||||
case Event(ClientDisconnected(name), d @ Data(clients, _, arrived, _)) ⇒
|
||||
if (clients.isEmpty) throw BarrierEmpty(d, "cannot disconnect " + name + ": no client to disconnect")
|
||||
(clients find (_.name == name)) match {
|
||||
case None ⇒ stay
|
||||
|
|
@ -520,38 +525,49 @@ private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoor
|
|||
}
|
||||
|
||||
when(Idle) {
|
||||
case Event(EnterBarrier(name), d @ Data(clients, _, _)) ⇒
|
||||
case Event(EnterBarrier(name, timeout), d @ Data(clients, _, _, _)) ⇒
|
||||
if (failed)
|
||||
stay replying ToClient(BarrierResult(name, false))
|
||||
else if (clients.map(_.fsm) == Set(sender))
|
||||
stay replying ToClient(BarrierResult(name, true))
|
||||
else if (clients.find(_.fsm == sender).isEmpty)
|
||||
stay replying ToClient(BarrierResult(name, false))
|
||||
else
|
||||
goto(Waiting) using d.copy(barrier = name, arrived = sender :: Nil)
|
||||
case Event(RemoveClient(name), d @ Data(clients, _, _)) ⇒
|
||||
else {
|
||||
goto(Waiting) using d.copy(barrier = name, arrived = sender :: Nil,
|
||||
deadline = getDeadline(timeout))
|
||||
}
|
||||
case Event(RemoveClient(name), d @ Data(clients, _, _, _)) ⇒
|
||||
if (clients.isEmpty) throw BarrierEmpty(d, "cannot remove " + name + ": no client to remove")
|
||||
stay using d.copy(clients = clients filterNot (_.name == name))
|
||||
}
|
||||
|
||||
onTransition {
|
||||
case Idle -> Waiting ⇒ setTimer("Timeout", StateTimeout, TestConductor().Settings.BarrierTimeout.duration, false)
|
||||
case Idle -> Waiting ⇒ setTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft, false)
|
||||
case Waiting -> Idle ⇒ cancelTimer("Timeout")
|
||||
}
|
||||
|
||||
when(Waiting) {
|
||||
case Event(EnterBarrier(name), d @ Data(clients, barrier, arrived)) ⇒
|
||||
case Event(EnterBarrier(name, timeout), d @ Data(clients, barrier, arrived, deadline)) ⇒
|
||||
if (name != barrier) throw WrongBarrier(name, sender, d)
|
||||
val together = if (clients.exists(_.fsm == sender)) sender :: arrived else arrived
|
||||
val enterDeadline = getDeadline(timeout)
|
||||
// we only allow the deadlines to get shorter
|
||||
if (enterDeadline < deadline) {
|
||||
setTimer("Timeout", StateTimeout, enterDeadline.timeLeft, false)
|
||||
handleBarrier(d.copy(arrived = together, deadline = enterDeadline))
|
||||
} else
|
||||
handleBarrier(d.copy(arrived = together))
|
||||
case Event(RemoveClient(name), d @ Data(clients, barrier, arrived)) ⇒
|
||||
case Event(RemoveClient(name), d @ Data(clients, barrier, arrived, _)) ⇒
|
||||
clients find (_.name == name) match {
|
||||
case None ⇒ stay
|
||||
case Some(client) ⇒
|
||||
handleBarrier(d.copy(clients = clients - client, arrived = arrived filterNot (_ == client.fsm)))
|
||||
}
|
||||
case Event(StateTimeout, data) ⇒
|
||||
throw BarrierTimeout(data)
|
||||
case Event(FailBarrier(name), d @ Data(_, barrier, _, _)) ⇒
|
||||
if (name != barrier) throw WrongBarrier(name, sender, d)
|
||||
throw FailedBarrier(d)
|
||||
case Event(StateTimeout, d) ⇒
|
||||
throw BarrierTimeout(d)
|
||||
}
|
||||
|
||||
initialize
|
||||
|
|
@ -568,5 +584,9 @@ private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoor
|
|||
}
|
||||
}
|
||||
|
||||
def getDeadline(timeout: Option[Duration]): Deadline = {
|
||||
Deadline.now + timeout.getOrElse(TestConductor().Settings.BarrierTimeout.duration)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,6 +10,8 @@ import akka.remote.testconductor.{ TestConductorProtocol ⇒ TCP }
|
|||
import com.google.protobuf.Message
|
||||
import akka.actor.Address
|
||||
import org.jboss.netty.handler.codec.oneone.OneToOneDecoder
|
||||
import akka.util.Duration
|
||||
import akka.remote.testconductor.TestConductorProtocol.BarrierOp
|
||||
|
||||
case class RoleName(name: String)
|
||||
|
||||
|
|
@ -28,7 +30,8 @@ private[akka] sealed trait ConfirmedClientOp extends ClientOp
|
|||
*/
|
||||
private[akka] case class Hello(name: String, addr: Address) extends NetworkOp
|
||||
|
||||
private[akka] case class EnterBarrier(name: String) extends ServerOp with NetworkOp
|
||||
private[akka] case class EnterBarrier(name: String, timeout: Option[Duration]) extends ServerOp with NetworkOp
|
||||
private[akka] case class FailBarrier(name: String) extends ServerOp with NetworkOp
|
||||
private[akka] case class BarrierResult(name: String, success: Boolean) extends UnconfirmedClientOp with NetworkOp
|
||||
|
||||
private[akka] case class Throttle(node: RoleName, target: RoleName, direction: Direction, rateMBit: Float) extends CommandOp
|
||||
|
|
@ -72,10 +75,16 @@ private[akka] class MsgEncoder extends OneToOneEncoder {
|
|||
x match {
|
||||
case Hello(name, addr) ⇒
|
||||
w.setHello(TCP.Hello.newBuilder.setName(name).setAddress(addr))
|
||||
case EnterBarrier(name) ⇒
|
||||
w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name))
|
||||
case EnterBarrier(name, timeout) ⇒
|
||||
val barrier = TCP.EnterBarrier.newBuilder.setName(name)
|
||||
timeout foreach (t ⇒ barrier.setTimeout(t.toNanos))
|
||||
barrier.setOp(BarrierOp.Enter)
|
||||
w.setBarrier(barrier)
|
||||
case BarrierResult(name, success) ⇒
|
||||
w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setStatus(success))
|
||||
val res = if (success) BarrierOp.Succeeded else BarrierOp.Failed
|
||||
w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setOp(res))
|
||||
case FailBarrier(name) ⇒
|
||||
w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setOp(BarrierOp.Fail))
|
||||
case ThrottleMsg(target, dir, rate) ⇒
|
||||
w.setFailure(TCP.InjectFailure.newBuilder.setAddress(target)
|
||||
.setFailure(TCP.FailType.Throttle).setDirection(dir).setRateMBit(rate))
|
||||
|
|
@ -114,8 +123,13 @@ private[akka] class MsgDecoder extends OneToOneDecoder {
|
|||
Hello(h.getName, h.getAddress)
|
||||
} else if (w.hasBarrier) {
|
||||
val barrier = w.getBarrier
|
||||
if (barrier.hasStatus) BarrierResult(barrier.getName, barrier.getStatus)
|
||||
else EnterBarrier(w.getBarrier.getName)
|
||||
barrier.getOp match {
|
||||
case BarrierOp.Succeeded ⇒ BarrierResult(barrier.getName, true)
|
||||
case BarrierOp.Failed ⇒ BarrierResult(barrier.getName, false)
|
||||
case BarrierOp.Fail ⇒ FailBarrier(barrier.getName)
|
||||
case BarrierOp.Enter ⇒ EnterBarrier(barrier.getName,
|
||||
if (barrier.hasTimeout) Option(Duration.fromNanos(barrier.getTimeout)) else None)
|
||||
}
|
||||
} else if (w.hasFailure) {
|
||||
val f = w.getFailure
|
||||
import TCP.{ FailType ⇒ FT }
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import com.typesafe.config.ConfigFactory
|
|||
import akka.util.Timeout
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import akka.pattern.{ ask, pipe }
|
||||
import akka.pattern.{ ask, pipe, AskTimeoutException }
|
||||
import akka.dispatch.Await
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.actor.Status
|
||||
|
|
@ -26,6 +26,7 @@ import org.jboss.netty.channel.WriteCompletionEvent
|
|||
import java.net.ConnectException
|
||||
import akka.util.Deadline
|
||||
import akka.actor.Scheduler
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
/**
|
||||
* The Player is the client component of the
|
||||
|
|
@ -76,10 +77,31 @@ trait Player { this: TestConductorExt ⇒
|
|||
* throw an exception in case of timeouts or other errors.
|
||||
*/
|
||||
def enter(name: String*) {
|
||||
enter(Settings.BarrierTimeout, name)
|
||||
}
|
||||
|
||||
/**
|
||||
* Enter the named barriers, one after the other, in the order given. Will
|
||||
* throw an exception in case of timeouts or other errors.
|
||||
*/
|
||||
def enter(timeout: Timeout, name: Seq[String]) {
|
||||
system.log.debug("entering barriers " + name.mkString("(", ", ", ")"))
|
||||
val stop = Deadline.now + timeout.duration
|
||||
name foreach { b ⇒
|
||||
import Settings.BarrierTimeout
|
||||
Await.result(client ? ToServer(EnterBarrier(b)), Duration.Inf)
|
||||
val barrierTimeout = stop.timeLeft
|
||||
if (barrierTimeout < Duration.Zero) {
|
||||
client ! ToServer(FailBarrier(b))
|
||||
throw new TimeoutException("Server timed out while waiting for barrier " + b);
|
||||
}
|
||||
try {
|
||||
implicit val timeout = Timeout(barrierTimeout + Settings.QueryTimeout.duration)
|
||||
Await.result(client ? ToServer(EnterBarrier(b, Option(barrierTimeout))), Duration.Inf)
|
||||
} catch {
|
||||
case e: AskTimeoutException ⇒
|
||||
client ! ToServer(FailBarrier(b))
|
||||
// Why don't TimeoutException have a constructor that takes a cause?
|
||||
throw new TimeoutException("Client timed out while waiting for barrier " + b);
|
||||
}
|
||||
system.log.debug("passed barrier {}", b)
|
||||
}
|
||||
}
|
||||
|
|
@ -88,7 +110,7 @@ trait Player { this: TestConductorExt ⇒
|
|||
* Query remote transport address of named node.
|
||||
*/
|
||||
def getAddressFor(name: RoleName): Future[Address] = {
|
||||
import Settings.BarrierTimeout
|
||||
import Settings.QueryTimeout
|
||||
client ? ToServer(GetAddress(name)) mapTo
|
||||
}
|
||||
}
|
||||
|
|
@ -168,7 +190,7 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress)
|
|||
case Event(ToServer(msg), d @ Data(Some(channel), None)) ⇒
|
||||
channel.write(msg)
|
||||
val token = msg match {
|
||||
case EnterBarrier(barrier) ⇒ barrier
|
||||
case EnterBarrier(barrier, timeout) ⇒ barrier
|
||||
case GetAddress(node) ⇒ node.name
|
||||
}
|
||||
stay using d.copy(runningOp = Some(token, sender))
|
||||
|
|
|
|||
|
|
@ -16,9 +16,9 @@ import org.jboss.netty.channel.ChannelPipelineFactory
|
|||
private[akka] class TestConductorTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider)
|
||||
extends NettyRemoteTransport(_system, _provider) {
|
||||
|
||||
override def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean): ChannelPipelineFactory =
|
||||
override def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory =
|
||||
new ChannelPipelineFactory {
|
||||
def getPipeline = PipelineFactory(new NetworkFailureInjector(system) +: PipelineFactory.defaultStack(withTimeout) :+ endpoint)
|
||||
def getPipeline = PipelineFactory(new NetworkFailureInjector(system) +: PipelineFactory.defaultStack(withTimeout, isClient) :+ endpoint)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -47,7 +47,7 @@ class LookupRemoteActorSpec extends MultiNodeSpec(LookupRemoteActorMultiJvmSpec)
|
|||
val masterAddress = testConductor.getAddressFor(master).await
|
||||
(hello ? "identify").await.asInstanceOf[ActorRef].path.address must equal(masterAddress)
|
||||
}
|
||||
testConductor.enter("done")
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -56,7 +56,7 @@ class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
|
|||
system.stop(actor)
|
||||
}
|
||||
|
||||
testConductor.enter("done")
|
||||
enterBarrier("done")
|
||||
}
|
||||
|
||||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef (with deployOnAll)" taggedAs LongRunningTest in {
|
||||
|
|
@ -74,7 +74,7 @@ class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec)
|
|||
system.stop(actor)
|
||||
}
|
||||
|
||||
testConductor.enter("done")
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -55,11 +55,11 @@ class RandomRoutedRemoteActorSpec extends MultiNodeSpec(RandomRoutedRemoteActorM
|
|||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first, second, third) {
|
||||
testConductor.enter("start", "broadcast-end", "end", "done")
|
||||
enterBarrier("start", "broadcast-end", "end", "done")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
testConductor.enter("start")
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(Props[SomeActor].withRouter(RandomRouter()), "service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
|
|
@ -76,17 +76,17 @@ class RandomRoutedRemoteActorSpec extends MultiNodeSpec(RandomRoutedRemoteActorM
|
|||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
testConductor.enter("broadcast-end")
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
testConductor.enter("end")
|
||||
enterBarrier("end")
|
||||
replies.values foreach { _ must be > (0) }
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
testConductor.enter("done")
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -55,11 +55,11 @@ class RoundRobinRoutedRemoteActorSpec extends MultiNodeSpec(RoundRobinRoutedRemo
|
|||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first, second, third) {
|
||||
testConductor.enter("start", "broadcast-end", "end", "done")
|
||||
enterBarrier("start", "broadcast-end", "end", "done")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
testConductor.enter("start")
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
|
|
@ -76,17 +76,17 @@ class RoundRobinRoutedRemoteActorSpec extends MultiNodeSpec(RoundRobinRoutedRemo
|
|||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
testConductor.enter("broadcast-end")
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
testConductor.enter("end")
|
||||
enterBarrier("end")
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
testConductor.enter("done")
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -55,11 +55,11 @@ class ScatterGatherRoutedRemoteActorSpec extends MultiNodeSpec(ScatterGatherRout
|
|||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first, second, third) {
|
||||
testConductor.enter("start", "broadcast-end", "end", "done")
|
||||
enterBarrier("start", "broadcast-end", "end", "done")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
testConductor.enter("start")
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(Props[SomeActor].withRouter(ScatterGatherFirstCompletedRouter(within = 10 seconds)), "service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
|
|
@ -76,17 +76,17 @@ class ScatterGatherRoutedRemoteActorSpec extends MultiNodeSpec(ScatterGatherRout
|
|||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
testConductor.enter("broadcast-end")
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
testConductor.enter("end")
|
||||
enterBarrier("end")
|
||||
replies.values.sum must be === connectionCount * iterationCount
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
testConductor.enter("done")
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -46,7 +46,7 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with Im
|
|||
}), "echo")
|
||||
}
|
||||
|
||||
testConductor.enter("name")
|
||||
enterBarrier("name")
|
||||
}
|
||||
|
||||
"support throttling of network connections" taggedAs LongRunningTest in {
|
||||
|
|
@ -62,7 +62,7 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with Im
|
|||
testConductor.throttle(slave, master, Direction.Send, rateMBit = 0.01).await
|
||||
}
|
||||
|
||||
testConductor.enter("throttled_send")
|
||||
enterBarrier("throttled_send")
|
||||
|
||||
runOn(slave) {
|
||||
for (i ← 0 to 9) echo ! i
|
||||
|
|
@ -73,14 +73,14 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with Im
|
|||
receiveN(9) must be(1 to 9)
|
||||
}
|
||||
|
||||
testConductor.enter("throttled_send2")
|
||||
enterBarrier("throttled_send2")
|
||||
|
||||
runOn(master) {
|
||||
testConductor.throttle(slave, master, Direction.Send, -1).await
|
||||
testConductor.throttle(slave, master, Direction.Receive, rateMBit = 0.01).await
|
||||
}
|
||||
|
||||
testConductor.enter("throttled_recv")
|
||||
enterBarrier("throttled_recv")
|
||||
|
||||
runOn(slave) {
|
||||
for (i ← 10 to 19) echo ! i
|
||||
|
|
@ -98,7 +98,7 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with Im
|
|||
receiveN(9) must be(11 to 19)
|
||||
}
|
||||
|
||||
testConductor.enter("throttled_recv2")
|
||||
enterBarrier("throttled_recv2")
|
||||
|
||||
runOn(master) {
|
||||
testConductor.throttle(slave, master, Direction.Receive, -1).await
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ import org.scalatest.BeforeAndAfterEach
|
|||
import java.net.InetSocketAddress
|
||||
import java.net.InetAddress
|
||||
import akka.testkit.TimingTest
|
||||
import akka.util.{ Timeout, Duration }
|
||||
|
||||
object BarrierSpec {
|
||||
case class Failed(ref: ActorRef, thr: Throwable)
|
||||
|
|
@ -31,7 +32,7 @@ object BarrierSpec {
|
|||
"""
|
||||
}
|
||||
|
||||
class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with BeforeAndAfterEach {
|
||||
class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender {
|
||||
|
||||
import BarrierSpec._
|
||||
import Controller._
|
||||
|
|
@ -41,10 +42,6 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
val B = RoleName("b")
|
||||
val C = RoleName("c")
|
||||
|
||||
override def afterEach {
|
||||
system.eventStream.setLogLevel(Logging.WarningLevel)
|
||||
}
|
||||
|
||||
"A BarrierCoordinator" must {
|
||||
|
||||
"register clients and remove them" taggedAs TimingTest in {
|
||||
|
|
@ -55,7 +52,7 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
EventFilter[BarrierEmpty](occurrences = 1) intercept {
|
||||
b ! RemoveClient(A)
|
||||
}
|
||||
expectMsg(Failed(b, BarrierEmpty(Data(Set(), "", Nil), "cannot remove RoleName(a): no client to remove")))
|
||||
expectMsg(Failed(b, BarrierEmpty(Data(Set(), "", Nil, null), "cannot remove RoleName(a): no client to remove")))
|
||||
}
|
||||
|
||||
"register clients and disconnect them" taggedAs TimingTest in {
|
||||
|
|
@ -65,17 +62,17 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
EventFilter[ClientLost](occurrences = 1) intercept {
|
||||
b ! ClientDisconnected(A)
|
||||
}
|
||||
expectMsg(Failed(b, ClientLost(Data(Set(), "", Nil), A)))
|
||||
expectMsg(Failed(b, ClientLost(Data(Set(), "", Nil, null), A)))
|
||||
EventFilter[BarrierEmpty](occurrences = 1) intercept {
|
||||
b ! ClientDisconnected(A)
|
||||
}
|
||||
expectMsg(Failed(b, BarrierEmpty(Data(Set(), "", Nil), "cannot disconnect RoleName(a): no client to disconnect")))
|
||||
expectMsg(Failed(b, BarrierEmpty(Data(Set(), "", Nil, null), "cannot disconnect RoleName(a): no client to disconnect")))
|
||||
}
|
||||
|
||||
"fail entering barrier when nobody registered" taggedAs TimingTest in {
|
||||
val b = getBarrier()
|
||||
b ! EnterBarrier("b")
|
||||
expectMsg(ToClient(BarrierResult("b", false)))
|
||||
b ! EnterBarrier("bar1", None)
|
||||
expectMsg(ToClient(BarrierResult("bar1", false)))
|
||||
}
|
||||
|
||||
"enter barrier" taggedAs TimingTest in {
|
||||
|
|
@ -83,12 +80,12 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
val a, b = TestProbe()
|
||||
barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar2", None))
|
||||
noMsg(a, b)
|
||||
within(2 second) {
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
a.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
within(2 seconds) {
|
||||
b.send(barrier, EnterBarrier("bar2", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar2", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar2", true)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -97,15 +94,15 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
val a, b, c = TestProbe()
|
||||
barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar3", None))
|
||||
barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref)
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
b.send(barrier, EnterBarrier("bar3", None))
|
||||
noMsg(a, b, c)
|
||||
within(2 second) {
|
||||
c.send(barrier, EnterBarrier("bar"))
|
||||
a.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
c.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
within(2 seconds) {
|
||||
c.send(barrier, EnterBarrier("bar3", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar3", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar3", true)))
|
||||
c.expectMsg(ToClient(BarrierResult("bar3", true)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -115,14 +112,14 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref)
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar4", None))
|
||||
b.send(barrier, EnterBarrier("bar4", None))
|
||||
barrier ! RemoveClient(A)
|
||||
barrier ! ClientDisconnected(A)
|
||||
noMsg(a, b, c)
|
||||
b.within(2 second) {
|
||||
b.within(2 seconds) {
|
||||
barrier ! RemoveClient(C)
|
||||
b.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar4", true)))
|
||||
}
|
||||
barrier ! ClientDisconnected(C)
|
||||
expectNoMsg(1 second)
|
||||
|
|
@ -133,9 +130,9 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
val a, b = TestProbe()
|
||||
barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar5", None))
|
||||
barrier ! RemoveClient(A)
|
||||
b.send(barrier, EnterBarrier("foo"))
|
||||
b.send(barrier, EnterBarrier("foo", None))
|
||||
b.expectMsg(ToClient(BarrierResult("foo", true)))
|
||||
}
|
||||
|
||||
|
|
@ -145,11 +142,15 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
barrier ! nodeA
|
||||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar6", None))
|
||||
EventFilter[ClientLost](occurrences = 1) intercept {
|
||||
barrier ! ClientDisconnected(B)
|
||||
}
|
||||
expectMsg(Failed(barrier, ClientLost(Data(Set(nodeA), "bar", a.ref :: Nil), B)))
|
||||
val msg = expectMsgType[Failed]
|
||||
msg match {
|
||||
case Failed(barrier, thr: ClientLost) if (thr == ClientLost(Data(Set(nodeA), "bar6", a.ref :: Nil, thr.data.deadline), B)) ⇒
|
||||
case x ⇒ fail("Expected " + Failed(barrier, ClientLost(Data(Set(nodeA), "bar6", a.ref :: Nil, null), B)) + " but got " + x)
|
||||
}
|
||||
}
|
||||
|
||||
"fail barrier with disconnecing node who already arrived" taggedAs TimingTest in {
|
||||
|
|
@ -160,12 +161,16 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! nodeA
|
||||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
barrier ! nodeC
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar7", None))
|
||||
b.send(barrier, EnterBarrier("bar7", None))
|
||||
EventFilter[ClientLost](occurrences = 1) intercept {
|
||||
barrier ! ClientDisconnected(B)
|
||||
}
|
||||
expectMsg(Failed(barrier, ClientLost(Data(Set(nodeA, nodeC), "bar", a.ref :: Nil), B)))
|
||||
val msg = expectMsgType[Failed]
|
||||
msg match {
|
||||
case Failed(barrier, thr: ClientLost) if (thr == ClientLost(Data(Set(nodeA, nodeC), "bar7", a.ref :: Nil, thr.data.deadline), B)) ⇒
|
||||
case x ⇒ fail("Expected " + Failed(barrier, ClientLost(Data(Set(nodeA, nodeC), "bar7", a.ref :: Nil, null), B)) + " but got " + x)
|
||||
}
|
||||
}
|
||||
|
||||
"fail when entering wrong barrier" taggedAs TimingTest in {
|
||||
|
|
@ -175,11 +180,15 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! nodeA
|
||||
val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
barrier ! nodeB
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar8", None))
|
||||
EventFilter[WrongBarrier](occurrences = 1) intercept {
|
||||
b.send(barrier, EnterBarrier("foo"))
|
||||
b.send(barrier, EnterBarrier("foo", None))
|
||||
}
|
||||
val msg = expectMsgType[Failed]
|
||||
msg match {
|
||||
case Failed(barrier, thr: WrongBarrier) if (thr == WrongBarrier("foo", b.ref, Data(Set(nodeA, nodeB), "bar8", a.ref :: Nil, thr.data.deadline))) ⇒
|
||||
case x ⇒ fail("Expected " + Failed(barrier, WrongBarrier("foo", b.ref, Data(Set(nodeA, nodeB), "bar8", a.ref :: Nil, null))) + " but got " + x)
|
||||
}
|
||||
expectMsg(Failed(barrier, WrongBarrier("foo", b.ref, Data(Set(nodeA, nodeB), "bar", a.ref :: Nil))))
|
||||
}
|
||||
|
||||
"fail barrier after first failure" taggedAs TimingTest in {
|
||||
|
|
@ -188,10 +197,14 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
EventFilter[BarrierEmpty](occurrences = 1) intercept {
|
||||
barrier ! RemoveClient(A)
|
||||
}
|
||||
expectMsg(Failed(barrier, BarrierEmpty(Data(Set(), "", Nil), "cannot remove RoleName(a): no client to remove")))
|
||||
val msg = expectMsgType[Failed]
|
||||
msg match {
|
||||
case Failed(barrier, thr: BarrierEmpty) if (thr == BarrierEmpty(Data(Set(), "", Nil, thr.data.deadline), "cannot remove RoleName(a): no client to remove")) ⇒
|
||||
case x ⇒ fail("Expected " + Failed(barrier, BarrierEmpty(Data(Set(), "", Nil, null), "cannot remove RoleName(a): no client to remove")) + " but got " + x)
|
||||
}
|
||||
barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
a.send(barrier, EnterBarrier("right"))
|
||||
a.expectMsg(ToClient(BarrierResult("right", false)))
|
||||
a.send(barrier, EnterBarrier("bar9", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar9", false)))
|
||||
}
|
||||
|
||||
"fail after barrier timeout" taggedAs TimingTest in {
|
||||
|
|
@ -201,9 +214,13 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
barrier ! nodeA
|
||||
barrier ! nodeB
|
||||
a.send(barrier, EnterBarrier("right"))
|
||||
a.send(barrier, EnterBarrier("bar10", None))
|
||||
EventFilter[BarrierTimeout](occurrences = 1) intercept {
|
||||
expectMsg(7 seconds, Failed(barrier, BarrierTimeout(Data(Set(nodeA, nodeB), "right", a.ref :: Nil))))
|
||||
val msg = expectMsgType[Failed](7 seconds)
|
||||
msg match {
|
||||
case Failed(barrier, thr: BarrierTimeout) if (thr == BarrierTimeout(Data(Set(nodeA, nodeB), "bar10", a.ref :: Nil, thr.data.deadline))) ⇒
|
||||
case x ⇒ fail("Expected " + Failed(barrier, BarrierTimeout(Data(Set(nodeA, nodeB), "bar10", a.ref :: Nil, null))) + " but got " + x)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -216,7 +233,11 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
EventFilter[DuplicateNode](occurrences = 1) intercept {
|
||||
barrier ! nodeB
|
||||
}
|
||||
expectMsg(Failed(barrier, DuplicateNode(Data(Set(nodeA), "", Nil), nodeB)))
|
||||
val msg = expectMsgType[Failed]
|
||||
msg match {
|
||||
case Failed(barrier, thr: DuplicateNode) if (thr == DuplicateNode(Data(Set(nodeA), "", Nil, thr.data.deadline), nodeB)) ⇒
|
||||
case x ⇒ fail("Expected " + Failed(barrier, DuplicateNode(Data(Set(nodeA), "", Nil, null), nodeB)) + " but got " + x)
|
||||
}
|
||||
}
|
||||
|
||||
"finally have no failure messages left" taggedAs TimingTest in {
|
||||
|
|
@ -253,7 +274,7 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
|
||||
"fail entering barrier when nobody registered" taggedAs TimingTest in {
|
||||
val b = getController(0)
|
||||
b ! EnterBarrier("b")
|
||||
b ! EnterBarrier("b", None)
|
||||
expectMsg(ToClient(BarrierResult("b", false)))
|
||||
}
|
||||
|
||||
|
|
@ -264,12 +285,12 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar11", None))
|
||||
noMsg(a, b)
|
||||
within(2 second) {
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
a.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
within(2 seconds) {
|
||||
b.send(barrier, EnterBarrier("bar11", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar11", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar11", true)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -280,16 +301,16 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar12", None))
|
||||
barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref)
|
||||
c.expectMsg(ToClient(Done))
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
b.send(barrier, EnterBarrier("bar12", None))
|
||||
noMsg(a, b, c)
|
||||
within(2 second) {
|
||||
c.send(barrier, EnterBarrier("bar"))
|
||||
a.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
c.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
within(2 seconds) {
|
||||
c.send(barrier, EnterBarrier("bar12", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar12", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar12", true)))
|
||||
c.expectMsg(ToClient(BarrierResult("bar12", true)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -302,14 +323,14 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
c.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar13", None))
|
||||
b.send(barrier, EnterBarrier("bar13", None))
|
||||
barrier ! Remove(A)
|
||||
barrier ! ClientDisconnected(A)
|
||||
noMsg(a, b, c)
|
||||
b.within(2 second) {
|
||||
b.within(2 seconds) {
|
||||
barrier ! Remove(C)
|
||||
b.expectMsg(ToClient(BarrierResult("bar", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar13", true)))
|
||||
}
|
||||
barrier ! ClientDisconnected(C)
|
||||
expectNoMsg(1 second)
|
||||
|
|
@ -322,9 +343,9 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar14", None))
|
||||
barrier ! Remove(A)
|
||||
b.send(barrier, EnterBarrier("foo"))
|
||||
b.send(barrier, EnterBarrier("foo", None))
|
||||
b.expectMsg(ToClient(BarrierResult("foo", true)))
|
||||
}
|
||||
|
||||
|
|
@ -336,13 +357,13 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar15", None))
|
||||
barrier ! ClientDisconnected(RoleName("unknown"))
|
||||
noMsg(a)
|
||||
EventFilter[ClientLost](occurrences = 1) intercept {
|
||||
barrier ! ClientDisconnected(B)
|
||||
}
|
||||
a.expectMsg(ToClient(BarrierResult("bar", false)))
|
||||
a.expectMsg(ToClient(BarrierResult("bar15", false)))
|
||||
}
|
||||
|
||||
"fail barrier with disconnecing node who already arrived" taggedAs TimingTest in {
|
||||
|
|
@ -356,12 +377,12 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
c.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
b.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar16", None))
|
||||
b.send(barrier, EnterBarrier("bar16", None))
|
||||
EventFilter[ClientLost](occurrences = 1) intercept {
|
||||
barrier ! ClientDisconnected(B)
|
||||
}
|
||||
a.expectMsg(ToClient(BarrierResult("bar", false)))
|
||||
a.expectMsg(ToClient(BarrierResult("bar16", false)))
|
||||
}
|
||||
|
||||
"fail when entering wrong barrier" taggedAs TimingTest in {
|
||||
|
|
@ -373,15 +394,15 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! nodeB
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar"))
|
||||
a.send(barrier, EnterBarrier("bar17", None))
|
||||
EventFilter[WrongBarrier](occurrences = 1) intercept {
|
||||
b.send(barrier, EnterBarrier("foo"))
|
||||
b.send(barrier, EnterBarrier("foo", None))
|
||||
}
|
||||
a.expectMsg(ToClient(BarrierResult("bar", false)))
|
||||
a.expectMsg(ToClient(BarrierResult("bar17", false)))
|
||||
b.expectMsg(ToClient(BarrierResult("foo", false)))
|
||||
}
|
||||
|
||||
"not really fail after barrier timeout" taggedAs TimingTest in {
|
||||
"fail after barrier timeout" taggedAs TimingTest in {
|
||||
val barrier = getController(2)
|
||||
val a, b = TestProbe()
|
||||
val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
|
|
@ -390,13 +411,13 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
barrier ! nodeB
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("right"))
|
||||
a.send(barrier, EnterBarrier("bar18", Option(2 seconds)))
|
||||
EventFilter[BarrierTimeout](occurrences = 1) intercept {
|
||||
Thread.sleep(5000)
|
||||
Thread.sleep(4000)
|
||||
}
|
||||
b.send(barrier, EnterBarrier("right"))
|
||||
a.expectMsg(ToClient(BarrierResult("right", true)))
|
||||
b.expectMsg(ToClient(BarrierResult("right", true)))
|
||||
b.send(barrier, EnterBarrier("bar18", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar18", false)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar18", false)))
|
||||
}
|
||||
|
||||
"fail if a node registers twice" taggedAs TimingTest in {
|
||||
|
|
@ -423,8 +444,75 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
controller ! nodeB
|
||||
b.expectMsg(ToClient(BarrierResult("initial startup", false)))
|
||||
}
|
||||
a.send(controller, EnterBarrier("x"))
|
||||
a.expectMsg(ToClient(BarrierResult("x", false)))
|
||||
a.send(controller, EnterBarrier("bar19", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar19", false)))
|
||||
}
|
||||
|
||||
"fail subsequent barriers after foreced failure" taggedAs TimingTest in {
|
||||
val barrier = getController(2)
|
||||
val a, b = TestProbe()
|
||||
val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
barrier ! nodeA
|
||||
barrier ! nodeB
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar20", Option(2 seconds)))
|
||||
EventFilter[FailedBarrier](occurrences = 1) intercept {
|
||||
b.send(barrier, FailBarrier("bar20"))
|
||||
a.expectMsg(ToClient(BarrierResult("bar20", false)))
|
||||
b.expectNoMsg(1 second)
|
||||
}
|
||||
a.send(barrier, EnterBarrier("bar21", None))
|
||||
b.send(barrier, EnterBarrier("bar21", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar21", false)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar21", false)))
|
||||
}
|
||||
|
||||
"timeout within the shortest timeout if the new timeout is shorter" taggedAs TimingTest in {
|
||||
val barrier = getController(3)
|
||||
val a, b, c = TestProbe()
|
||||
val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
val nodeC = NodeInfo(C, AddressFromURIString("akka://sys"), c.ref)
|
||||
barrier ! nodeA
|
||||
barrier ! nodeB
|
||||
barrier ! nodeC
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
c.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar22", Option(10 seconds)))
|
||||
b.send(barrier, EnterBarrier("bar22", Option(2 seconds)))
|
||||
EventFilter[BarrierTimeout](occurrences = 1) intercept {
|
||||
Thread.sleep(4000)
|
||||
}
|
||||
c.send(barrier, EnterBarrier("bar22", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar22", false)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar22", false)))
|
||||
c.expectMsg(ToClient(BarrierResult("bar22", false)))
|
||||
}
|
||||
|
||||
"timeout within the shortest timeout if the new timeout is longer" taggedAs TimingTest in {
|
||||
val barrier = getController(3)
|
||||
val a, b, c = TestProbe()
|
||||
val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref)
|
||||
val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref)
|
||||
val nodeC = NodeInfo(C, AddressFromURIString("akka://sys"), c.ref)
|
||||
barrier ! nodeA
|
||||
barrier ! nodeB
|
||||
barrier ! nodeC
|
||||
a.expectMsg(ToClient(Done))
|
||||
b.expectMsg(ToClient(Done))
|
||||
c.expectMsg(ToClient(Done))
|
||||
a.send(barrier, EnterBarrier("bar23", Option(2 seconds)))
|
||||
b.send(barrier, EnterBarrier("bar23", Option(10 seconds)))
|
||||
EventFilter[BarrierTimeout](occurrences = 1) intercept {
|
||||
Thread.sleep(4000)
|
||||
}
|
||||
c.send(barrier, EnterBarrier("bar23", None))
|
||||
a.expectMsg(ToClient(BarrierResult("bar23", false)))
|
||||
b.expectMsg(ToClient(BarrierResult("bar23", false)))
|
||||
c.expectMsg(ToClient(BarrierResult("bar23", false)))
|
||||
}
|
||||
|
||||
"finally have no failure messages left" taggedAs TimingTest in {
|
||||
|
|
@ -469,4 +557,7 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender with
|
|||
probes foreach (_.msgAvailable must be(false))
|
||||
}
|
||||
|
||||
private def data(clients: Set[Controller.NodeInfo], barrier: String, arrived: List[ActorRef], previous: Data): Data = {
|
||||
Data(clients, barrier, arrived, previous.deadline)
|
||||
}
|
||||
}
|
||||
|
|
@ -12,7 +12,7 @@ import akka.dispatch.Await
|
|||
import akka.dispatch.Await.Awaitable
|
||||
import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName }
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.util.{ NonFatal, Duration }
|
||||
import akka.util.{ Timeout, NonFatal, Duration }
|
||||
|
||||
/**
|
||||
* Configure the role names and participants of the test, including configuration settings.
|
||||
|
|
@ -182,6 +182,14 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
|
|||
if (nodes exists (_ == myself)) yes else no
|
||||
}
|
||||
|
||||
/**
|
||||
* Enter the named barriers in the order given. Use the remaining duration from
|
||||
* the innermost enclosing `within` block or the default `BarrierTimeout`
|
||||
*/
|
||||
def enterBarrier(name: String*) {
|
||||
testConductor.enter(Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)), name)
|
||||
}
|
||||
|
||||
/**
|
||||
* Query the controller for the transport address of the given node (by role name) and
|
||||
* return that as an ActorPath for easy composition:
|
||||
|
|
@ -193,11 +201,12 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
|
|||
def node(role: RoleName): ActorPath = RootActorPath(testConductor.getAddressFor(role).await)
|
||||
|
||||
/**
|
||||
* Enrich `.await()` onto all Awaitables, using BarrierTimeout.
|
||||
* Enrich `.await()` onto all Awaitables, using remaining duration from the innermost
|
||||
* enclosing `within` block or QueryTimeout.
|
||||
*/
|
||||
implicit def awaitHelper[T](w: Awaitable[T]) = new AwaitHelper(w)
|
||||
class AwaitHelper[T](w: Awaitable[T]) {
|
||||
def await: T = Await.result(w, testConductor.Settings.BarrierTimeout.duration)
|
||||
def await: T = Await.result(w, remainingOr(testConductor.Settings.QueryTimeout.duration))
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
@ -206,9 +215,11 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
|
|||
|
||||
private val controllerAddr = new InetSocketAddress(nodeNames(0), 4711)
|
||||
if (selfIndex == 0) {
|
||||
testConductor.startController(initialParticipants, myself, controllerAddr).await
|
||||
Await.result(testConductor.startController(initialParticipants, myself, controllerAddr),
|
||||
testConductor.Settings.BarrierTimeout.duration)
|
||||
} else {
|
||||
testConductor.startClient(myself, controllerAddr).await
|
||||
Await.result(testConductor.startClient(myself, controllerAddr),
|
||||
testConductor.Settings.BarrierTimeout.duration)
|
||||
}
|
||||
|
||||
// now add deployments, if so desired
|
||||
|
|
|
|||
|
|
@ -9,6 +9,8 @@ option optimize_for = SPEED;
|
|||
Compile with:
|
||||
cd ./akka-remote/src/main/protocol
|
||||
protoc RemoteProtocol.proto --java_out ../java
|
||||
cd ../../../..
|
||||
./scripts/fix-protobuf.sh
|
||||
*******************************************/
|
||||
|
||||
message AkkaRemoteProtocol {
|
||||
|
|
|
|||
|
|
@ -165,6 +165,52 @@ akka {
|
|||
|
||||
# (O) Maximum time window that a client should try to reconnect for
|
||||
reconnection-time-window = 600s
|
||||
|
||||
ssl {
|
||||
# (I&O) Enable SSL/TLS encryption.
|
||||
# This must be enabled on both the client and server to work.
|
||||
enable = off
|
||||
|
||||
# (I) This is the Java Key Store used by the server connection
|
||||
key-store = "keystore"
|
||||
|
||||
# This password is used for decrypting the key store
|
||||
key-store-password = "changeme"
|
||||
|
||||
# (O) This is the Java Key Store used by the client connection
|
||||
trust-store = "truststore"
|
||||
|
||||
# This password is used for decrypting the trust store
|
||||
trust-store-password = "changeme"
|
||||
|
||||
# (I&O) Protocol to use for SSL encryption, choose from:
|
||||
# Java 6 & 7:
|
||||
# 'SSLv3', 'TLSv1'
|
||||
# Java 7:
|
||||
# 'TLSv1.1', 'TLSv1.2'
|
||||
protocol = "TLSv1"
|
||||
|
||||
# Examples: [ "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA" ]
|
||||
# You need to install the JCE Unlimited Strength Jurisdiction Policy Files to use AES 256
|
||||
# More info here: http://docs.oracle.com/javase/7/docs/technotes/guides/security/SunProviders.html#SunJCEProvider
|
||||
enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA"]
|
||||
|
||||
# Using /dev/./urandom is only necessary when using SHA1PRNG on Linux to prevent blocking
|
||||
# It is NOT as secure because it reuses the seed
|
||||
# '' => defaults to /dev/random or whatever is set in java.security for example: securerandom.source=file:/dev/random
|
||||
# '/dev/./urandom' => NOT '/dev/urandom' as that doesn't work according to: http://bugs.sun.com/view_bug.do?bug_id=6202721
|
||||
sha1prng-random-source = ""
|
||||
|
||||
# There are three options, in increasing order of security:
|
||||
# "" or SecureRandom => (default)
|
||||
# "SHA1PRNG" => Can be slow because of blocking issues on Linux
|
||||
# "AES128CounterRNGFast" => fastest startup and based on AES encryption algorithm
|
||||
# The following use one of 3 possible seed sources, depending on availability: /dev/random, random.org and SecureRandom (provided by Java)
|
||||
# "AES128CounterRNGSecure"
|
||||
# "AES256CounterRNGSecure" (Install JCE Unlimited Strength Jurisdiction Policy Files first)
|
||||
# Setting a value here may require you to supply the appropriate cipher suite (see enabled-algorithms section above)
|
||||
random-number-generator = ""
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -145,7 +145,7 @@ private[akka] class ActiveRemoteClient private[akka] (
|
|||
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
|
||||
|
||||
val b = new ClientBootstrap(netty.clientChannelFactory)
|
||||
b.setPipelineFactory(netty.createPipeline(new ActiveRemoteClientHandler(name, b, remoteAddress, localAddress, netty.timer, this), true))
|
||||
b.setPipelineFactory(netty.createPipeline(new ActiveRemoteClientHandler(name, b, remoteAddress, localAddress, netty.timer, this), withTimeout = true, isClient = true))
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("keepAlive", true)
|
||||
b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis)
|
||||
|
|
|
|||
|
|
@ -64,16 +64,17 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
*
|
||||
* @param withTimeout determines whether an IdleStateHandler shall be included
|
||||
*/
|
||||
def apply(endpoint: ⇒ Seq[ChannelHandler], withTimeout: Boolean): ChannelPipelineFactory =
|
||||
def apply(endpoint: ⇒ Seq[ChannelHandler], withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory =
|
||||
new ChannelPipelineFactory {
|
||||
def getPipeline = apply(defaultStack(withTimeout) ++ endpoint)
|
||||
def getPipeline = apply(defaultStack(withTimeout, isClient) ++ endpoint)
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct a default protocol stack, excluding the “head” handler (i.e. the one which
|
||||
* actually dispatches the received messages to the local target actors).
|
||||
*/
|
||||
def defaultStack(withTimeout: Boolean): Seq[ChannelHandler] =
|
||||
def defaultStack(withTimeout: Boolean, isClient: Boolean): Seq[ChannelHandler] =
|
||||
(if (settings.EnableSSL) NettySSLSupport(settings, NettyRemoteTransport.this.log, isClient) :: Nil else Nil) :::
|
||||
(if (withTimeout) timeout :: Nil else Nil) :::
|
||||
msgFormat :::
|
||||
authenticator :::
|
||||
|
|
@ -122,8 +123,8 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
* This method is factored out to provide an extension point in case the
|
||||
* pipeline shall be changed. It is recommended to use
|
||||
*/
|
||||
def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean): ChannelPipelineFactory =
|
||||
PipelineFactory(Seq(endpoint), withTimeout)
|
||||
def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory =
|
||||
PipelineFactory(Seq(endpoint), withTimeout, isClient)
|
||||
|
||||
private val remoteClients = new HashMap[Address, RemoteClient]
|
||||
private val clientsLock = new ReentrantReadWriteLock
|
||||
|
|
|
|||
|
|
@ -0,0 +1,134 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote.netty
|
||||
|
||||
import org.jboss.netty.handler.ssl.SslHandler
|
||||
import javax.net.ssl.{ KeyManagerFactory, TrustManager, TrustManagerFactory, SSLContext }
|
||||
import akka.remote.RemoteTransportException
|
||||
import akka.event.LoggingAdapter
|
||||
import java.io.{ IOException, FileNotFoundException, FileInputStream }
|
||||
import java.security.{ SecureRandom, GeneralSecurityException, KeyStore, Security }
|
||||
import akka.security.provider.AkkaProvider
|
||||
|
||||
/**
|
||||
* Used for adding SSL support to Netty pipeline
|
||||
* Internal use only
|
||||
*/
|
||||
private[akka] object NettySSLSupport {
|
||||
|
||||
val akka = new AkkaProvider
|
||||
Security.addProvider(akka)
|
||||
|
||||
/**
|
||||
* Construct a SSLHandler which can be inserted into a Netty server/client pipeline
|
||||
*/
|
||||
def apply(settings: NettySettings, log: LoggingAdapter, isClient: Boolean): SslHandler =
|
||||
if (isClient) initializeClientSSL(settings, log) else initializeServerSSL(settings, log)
|
||||
|
||||
def initializeCustomSecureRandom(rngName: Option[String], sourceOfRandomness: Option[String], log: LoggingAdapter): SecureRandom = {
|
||||
/**
|
||||
* According to this bug report: http://bugs.sun.com/view_bug.do?bug_id=6202721
|
||||
* Using /dev/./urandom is only necessary when using SHA1PRNG on Linux
|
||||
* <quote>Use 'new SecureRandom()' instead of 'SecureRandom.getInstance("SHA1PRNG")'</quote> to avoid having problems
|
||||
*/
|
||||
sourceOfRandomness foreach { path ⇒ System.setProperty("java.security.egd", path) }
|
||||
|
||||
val rng = rngName match {
|
||||
case Some(r @ ("AES128CounterRNGFast" | "AES128CounterRNGSecure" | "AES256CounterRNGSecure")) ⇒
|
||||
log.debug("SSL random number generator set to: {}", r)
|
||||
SecureRandom.getInstance(r, akka)
|
||||
case Some("SHA1PRNG") ⇒
|
||||
log.debug("SSL random number generator set to: SHA1PRNG")
|
||||
// This needs /dev/urandom to be the source on Linux to prevent problems with /dev/random blocking
|
||||
// However, this also makes the seed source insecure as the seed is reused to avoid blocking (not a problem on FreeBSD).
|
||||
SecureRandom.getInstance("SHA1PRNG")
|
||||
case Some(unknown) ⇒
|
||||
log.debug("Unknown SSLRandomNumberGenerator [{}] falling back to SecureRandom", unknown)
|
||||
new SecureRandom
|
||||
case None ⇒
|
||||
log.debug("SSLRandomNumberGenerator not specified, falling back to SecureRandom")
|
||||
new SecureRandom
|
||||
}
|
||||
rng.nextInt() // prevent stall on first access
|
||||
rng
|
||||
}
|
||||
|
||||
def initializeClientSSL(settings: NettySettings, log: LoggingAdapter): SslHandler = {
|
||||
log.debug("Client SSL is enabled, initialising ...")
|
||||
|
||||
def constructClientContext(settings: NettySettings, log: LoggingAdapter, trustStorePath: String, trustStorePassword: String, protocol: String): Option[SSLContext] =
|
||||
try {
|
||||
val trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm)
|
||||
val trustStore = KeyStore.getInstance(KeyStore.getDefaultType)
|
||||
trustStore.load(new FileInputStream(trustStorePath), trustStorePassword.toCharArray) //FIXME does the FileInputStream need to be closed?
|
||||
trustManagerFactory.init(trustStore)
|
||||
val trustManagers: Array[TrustManager] = trustManagerFactory.getTrustManagers
|
||||
Option(SSLContext.getInstance(protocol)) map { ctx ⇒ ctx.init(null, trustManagers, initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, settings.SSLRandomSource, log)); ctx }
|
||||
} catch {
|
||||
case e: FileNotFoundException ⇒ throw new RemoteTransportException("Client SSL connection could not be established because trust store could not be loaded", e)
|
||||
case e: IOException ⇒ throw new RemoteTransportException("Client SSL connection could not be established because: " + e.getMessage, e)
|
||||
case e: GeneralSecurityException ⇒ throw new RemoteTransportException("Client SSL connection could not be established because SSL context could not be constructed", e)
|
||||
}
|
||||
|
||||
((settings.SSLTrustStore, settings.SSLTrustStorePassword, settings.SSLProtocol) match {
|
||||
case (Some(trustStore), Some(password), Some(protocol)) ⇒ constructClientContext(settings, log, trustStore, password, protocol)
|
||||
case (trustStore, password, protocol) ⇒ throw new GeneralSecurityException(
|
||||
"One or several SSL trust store settings are missing: [trust-store: %s] [trust-store-password: %s] [protocol: %s]".format(
|
||||
trustStore,
|
||||
password,
|
||||
protocol))
|
||||
}) match {
|
||||
case Some(context) ⇒
|
||||
log.debug("Using client SSL context to create SSLEngine ...")
|
||||
val sslEngine = context.createSSLEngine
|
||||
sslEngine.setUseClientMode(true)
|
||||
sslEngine.setEnabledCipherSuites(settings.SSLEnabledAlgorithms.toArray.map(_.toString))
|
||||
new SslHandler(sslEngine)
|
||||
case None ⇒
|
||||
throw new GeneralSecurityException(
|
||||
"""Failed to initialize client SSL because SSL context could not be found." +
|
||||
"Make sure your settings are correct: [trust-store: %s] [trust-store-password: %s] [protocol: %s]""".format(
|
||||
settings.SSLTrustStore,
|
||||
settings.SSLTrustStorePassword,
|
||||
settings.SSLProtocol))
|
||||
}
|
||||
}
|
||||
|
||||
def initializeServerSSL(settings: NettySettings, log: LoggingAdapter): SslHandler = {
|
||||
log.debug("Server SSL is enabled, initialising ...")
|
||||
|
||||
def constructServerContext(settings: NettySettings, log: LoggingAdapter, keyStorePath: String, keyStorePassword: String, protocol: String): Option[SSLContext] =
|
||||
try {
|
||||
val factory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm)
|
||||
val keyStore = KeyStore.getInstance(KeyStore.getDefaultType)
|
||||
keyStore.load(new FileInputStream(keyStorePath), keyStorePassword.toCharArray) //FIXME does the FileInputStream need to be closed?
|
||||
factory.init(keyStore, keyStorePassword.toCharArray)
|
||||
Option(SSLContext.getInstance(protocol)) map { ctx ⇒ ctx.init(factory.getKeyManagers, null, initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, settings.SSLRandomSource, log)); ctx }
|
||||
} catch {
|
||||
case e: FileNotFoundException ⇒ throw new RemoteTransportException("Server SSL connection could not be established because key store could not be loaded", e)
|
||||
case e: IOException ⇒ throw new RemoteTransportException("Server SSL connection could not be established because: " + e.getMessage, e)
|
||||
case e: GeneralSecurityException ⇒ throw new RemoteTransportException("Server SSL connection could not be established because SSL context could not be constructed", e)
|
||||
}
|
||||
|
||||
((settings.SSLKeyStore, settings.SSLKeyStorePassword, settings.SSLProtocol) match {
|
||||
case (Some(keyStore), Some(password), Some(protocol)) ⇒ constructServerContext(settings, log, keyStore, password, protocol)
|
||||
case (keyStore, password, protocol) ⇒ throw new GeneralSecurityException(
|
||||
"SSL key store settings went missing. [key-store: %s] [key-store-password: %s] [protocol: %s]".format(keyStore, password, protocol))
|
||||
}) match {
|
||||
case Some(context) ⇒
|
||||
log.debug("Using server SSL context to create SSLEngine ...")
|
||||
val sslEngine = context.createSSLEngine
|
||||
sslEngine.setUseClientMode(false)
|
||||
sslEngine.setEnabledCipherSuites(settings.SSLEnabledAlgorithms.toArray.map(_.toString))
|
||||
new SslHandler(sslEngine)
|
||||
case None ⇒ throw new GeneralSecurityException(
|
||||
"""Failed to initialize server SSL because SSL context could not be found.
|
||||
Make sure your settings are correct: [key-store: %s] [key-store-password: %s] [protocol: %s]""".format(
|
||||
settings.SSLKeyStore,
|
||||
settings.SSLKeyStorePassword,
|
||||
settings.SSLProtocol))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -12,7 +12,6 @@ import org.jboss.netty.channel.group.ChannelGroup
|
|||
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.execution.ExecutionHandler
|
||||
import akka.event.Logging
|
||||
import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol }
|
||||
import akka.remote.{ RemoteServerShutdown, RemoteServerError, RemoteServerClientDisconnected, RemoteServerClientConnected, RemoteServerClientClosed, RemoteProtocol, RemoteMessage }
|
||||
import akka.actor.Address
|
||||
|
|
@ -40,7 +39,7 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) {
|
|||
|
||||
private val bootstrap = {
|
||||
val b = new ServerBootstrap(factory)
|
||||
b.setPipelineFactory(netty.createPipeline(new RemoteServerHandler(openChannels, netty), false))
|
||||
b.setPipelineFactory(netty.createPipeline(new RemoteServerHandler(openChannels, netty), withTimeout = false, isClient = false))
|
||||
b.setOption("backlog", settings.Backlog)
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("child.keepAlive", true)
|
||||
|
|
|
|||
|
|
@ -86,4 +86,55 @@ private[akka] class NettySettings(config: Config, val systemName: String) {
|
|||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
val SSLKeyStore = getString("ssl.key-store") match {
|
||||
case "" ⇒ None
|
||||
case keyStore ⇒ Some(keyStore)
|
||||
}
|
||||
|
||||
val SSLTrustStore = getString("ssl.trust-store") match {
|
||||
case "" ⇒ None
|
||||
case trustStore ⇒ Some(trustStore)
|
||||
}
|
||||
|
||||
val SSLKeyStorePassword = getString("ssl.key-store-password") match {
|
||||
case "" ⇒ None
|
||||
case password ⇒ Some(password)
|
||||
}
|
||||
|
||||
val SSLTrustStorePassword = getString("ssl.trust-store-password") match {
|
||||
case "" ⇒ None
|
||||
case password ⇒ Some(password)
|
||||
}
|
||||
|
||||
val SSLEnabledAlgorithms = getStringList("ssl.enabled-algorithms").toArray.toSet
|
||||
|
||||
val SSLProtocol = getString("ssl.protocol") match {
|
||||
case "" ⇒ None
|
||||
case protocol ⇒ Some(protocol)
|
||||
}
|
||||
|
||||
val SSLRandomSource = getString("ssl.sha1prng-random-source") match {
|
||||
case "" ⇒ None
|
||||
case path ⇒ Some(path)
|
||||
}
|
||||
|
||||
val SSLRandomNumberGenerator = getString("ssl.random-number-generator") match {
|
||||
case "" ⇒ None
|
||||
case rng ⇒ Some(rng)
|
||||
}
|
||||
|
||||
val EnableSSL = {
|
||||
val enableSSL = getBoolean("ssl.enable")
|
||||
if (enableSSL) {
|
||||
if (SSLProtocol.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.netty.ssl.enable is turned on but no protocol is defined in 'akka.remote.netty.ssl.protocol'.")
|
||||
if (SSLKeyStore.isEmpty && SSLTrustStore.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.netty.ssl.enable is turned on but no key/trust store is defined in 'akka.remote.netty.ssl.key-store' / 'akka.remote.netty.ssl.trust-store'.")
|
||||
if (SSLKeyStore.isDefined && SSLKeyStorePassword.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.netty.ssl.key-store' is defined but no key-store password is defined in 'akka.remote.netty.ssl.key-store-password'.")
|
||||
if (SSLTrustStore.isDefined && SSLTrustStorePassword.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.netty.ssl.trust-store' is defined but no trust-store password is defined in 'akka.remote.netty.ssl.trust-store-password'.")
|
||||
}
|
||||
enableSSL
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,36 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.security.provider
|
||||
|
||||
import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator }
|
||||
import java.security.SecureRandom
|
||||
|
||||
/**
|
||||
* Internal API
|
||||
*/
|
||||
class AES128CounterRNGFast extends java.security.SecureRandomSpi {
|
||||
private val rng = new AESCounterRNG(new SecureRandomSeedGenerator())
|
||||
|
||||
/**
|
||||
* This is managed internally only
|
||||
*/
|
||||
override protected def engineSetSeed(seed: Array[Byte]): Unit = ()
|
||||
|
||||
/**
|
||||
* Generates a user-specified number of random bytes.
|
||||
*
|
||||
* @param bytes the array to be filled in with random bytes.
|
||||
*/
|
||||
override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes)
|
||||
|
||||
/**
|
||||
* Returns the given number of seed bytes. This call may be used to
|
||||
* seed other random number generators.
|
||||
*
|
||||
* @param numBytes the number of seed bytes to generate.
|
||||
* @return the seed bytes.
|
||||
*/
|
||||
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = (new SecureRandom).generateSeed(numBytes)
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.security.provider
|
||||
|
||||
import org.uncommons.maths.random.{ AESCounterRNG, DefaultSeedGenerator }
|
||||
|
||||
/**
|
||||
* Internal API
|
||||
*/
|
||||
class AES128CounterRNGSecure extends java.security.SecureRandomSpi {
|
||||
private val rng = new AESCounterRNG()
|
||||
|
||||
/**
|
||||
* This is managed internally only
|
||||
*/
|
||||
override protected def engineSetSeed(seed: Array[Byte]): Unit = ()
|
||||
|
||||
/**
|
||||
* Generates a user-specified number of random bytes.
|
||||
*
|
||||
* @param bytes the array to be filled in with random bytes.
|
||||
*/
|
||||
override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes)
|
||||
|
||||
/**
|
||||
* Returns the given number of seed bytes. This call may be used to
|
||||
* seed other random number generators.
|
||||
*
|
||||
* @param numBytes the number of seed bytes to generate.
|
||||
* @return the seed bytes.
|
||||
*/
|
||||
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = DefaultSeedGenerator.getInstance.generateSeed(numBytes)
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.security.provider
|
||||
|
||||
import org.uncommons.maths.random.{ AESCounterRNG, DefaultSeedGenerator }
|
||||
|
||||
/**
|
||||
* Internal API
|
||||
*/
|
||||
class AES256CounterRNGSecure extends java.security.SecureRandomSpi {
|
||||
private val rng = new AESCounterRNG(32) // Magic number is magic
|
||||
|
||||
/**
|
||||
* This is managed internally only
|
||||
*/
|
||||
override protected def engineSetSeed(seed: Array[Byte]): Unit = ()
|
||||
|
||||
/**
|
||||
* Generates a user-specified number of random bytes.
|
||||
*
|
||||
* @param bytes the array to be filled in with random bytes.
|
||||
*/
|
||||
override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes)
|
||||
|
||||
/**
|
||||
* Returns the given number of seed bytes. This call may be used to
|
||||
* seed other random number generators.
|
||||
*
|
||||
* @param numBytes the number of seed bytes to generate.
|
||||
* @return the seed bytes.
|
||||
*/
|
||||
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = DefaultSeedGenerator.getInstance.generateSeed(numBytes)
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.security.provider
|
||||
|
||||
import java.security.{ PrivilegedAction, AccessController, Provider }
|
||||
|
||||
/**
|
||||
* A provider that for AES128CounterRNGFast, a cryptographically secure random number generator through SecureRandom
|
||||
*/
|
||||
final class AkkaProvider extends Provider("Akka", 1.0, "Akka provider 1.0 that implements a secure AES random number generator") {
|
||||
AccessController.doPrivileged(new PrivilegedAction[AkkaProvider] {
|
||||
def run = {
|
||||
//SecureRandom
|
||||
put("SecureRandom.AES128CounterRNGFast", "akka.security.provider.AES128CounterRNGFast")
|
||||
put("SecureRandom.AES128CounterRNGSecure", "akka.security.provider.AES128CounterRNGSecure")
|
||||
put("SecureRandom.AES256CounterRNGSecure", "akka.security.provider.AES256CounterRNGSecure")
|
||||
|
||||
//Implementation type: software or hardware
|
||||
put("SecureRandom.AES128CounterRNGFast ImplementedIn", "Software")
|
||||
put("SecureRandom.AES128CounterRNGSecure ImplementedIn", "Software")
|
||||
put("SecureRandom.AES256CounterRNGSecure ImplementedIn", "Software")
|
||||
null //Magic null is magic
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
|
|
@ -4,11 +4,13 @@
|
|||
|
||||
package akka.actor;
|
||||
|
||||
/*
|
||||
/******************************************
|
||||
Compile with:
|
||||
cd ./akka-remote/src/test/protocol
|
||||
protoc ProtobufProtocol.proto --java_out ../java
|
||||
*/
|
||||
cd ../../../..
|
||||
./scripts/fix-protobuf.sh
|
||||
*******************************************/
|
||||
|
||||
message MyMessage {
|
||||
required uint64 id = 1;
|
||||
|
|
|
|||
BIN
akka-remote/src/test/resources/keystore
Normal file
BIN
akka-remote/src/test/resources/keystore
Normal file
Binary file not shown.
BIN
akka-remote/src/test/resources/truststore
Normal file
BIN
akka-remote/src/test/resources/truststore
Normal file
Binary file not shown.
|
|
@ -0,0 +1,205 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote
|
||||
|
||||
import akka.testkit._
|
||||
import akka.actor._
|
||||
import com.typesafe.config._
|
||||
import akka.dispatch.{ Await, Future }
|
||||
import akka.pattern.ask
|
||||
import java.io.File
|
||||
import akka.event.{ NoLogging, LoggingAdapter }
|
||||
import java.security.{ NoSuchAlgorithmException, SecureRandom, PrivilegedAction, AccessController }
|
||||
import netty.{ NettySettings, NettySSLSupport }
|
||||
import javax.net.ssl.SSLException
|
||||
import akka.util.{ Timeout, Duration }
|
||||
import akka.util.duration._
|
||||
|
||||
object Configuration {
|
||||
// set this in your JAVA_OPTS to see all ssl debug info: "-Djavax.net.debug=ssl,keymanager"
|
||||
// The certificate will expire in 2109
|
||||
private val trustStore = getClass.getClassLoader.getResource("truststore").getPath
|
||||
private val keyStore = getClass.getClassLoader.getResource("keystore").getPath
|
||||
private val conf = """
|
||||
akka {
|
||||
actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
test {
|
||||
single-expect-default = 10s
|
||||
filter-leeway = 10s
|
||||
default-timeout = 10s
|
||||
}
|
||||
|
||||
remote.netty {
|
||||
hostname = localhost
|
||||
ssl {
|
||||
enable = on
|
||||
trust-store = "%s"
|
||||
key-store = "%s"
|
||||
random-number-generator = "%s"
|
||||
enabled-algorithms = [%s]
|
||||
sha1prng-random-source = "/dev/./urandom"
|
||||
}
|
||||
}
|
||||
actor.deployment {
|
||||
/blub.remote = "akka://remote-sys@localhost:12346"
|
||||
/looker/child.remote = "akka://remote-sys@localhost:12346"
|
||||
/looker/child/grandchild.remote = "akka://Ticket1978CommunicationSpec@localhost:12345"
|
||||
}
|
||||
}
|
||||
"""
|
||||
|
||||
def getCipherConfig(cipher: String, enabled: String*): (String, Boolean, Config) = try {
|
||||
|
||||
if (true) throw new IllegalArgumentException("Ticket1978*Spec isn't enabled")
|
||||
|
||||
val config = ConfigFactory.parseString("akka.remote.netty.port=12345").withFallback(ConfigFactory.parseString(conf.format(trustStore, keyStore, cipher, enabled.mkString(", "))))
|
||||
val fullConfig = config.withFallback(AkkaSpec.testConf).withFallback(ConfigFactory.load).getConfig("akka.remote.netty")
|
||||
val settings = new NettySettings(fullConfig, "placeholder")
|
||||
|
||||
val rng = NettySSLSupport.initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, settings.SSLRandomSource, NoLogging)
|
||||
|
||||
rng.nextInt() // Has to work
|
||||
settings.SSLRandomNumberGenerator foreach { sRng ⇒ rng.getAlgorithm == sRng || (throw new NoSuchAlgorithmException(sRng)) }
|
||||
|
||||
val engine = NettySSLSupport.initializeClientSSL(settings, NoLogging).getEngine
|
||||
val gotAllSupported = enabled.toSet -- engine.getSupportedCipherSuites.toSet
|
||||
val gotAllEnabled = enabled.toSet -- engine.getEnabledCipherSuites.toSet
|
||||
gotAllSupported.isEmpty || (throw new IllegalArgumentException("Cipher Suite not supported: " + gotAllSupported))
|
||||
gotAllEnabled.isEmpty || (throw new IllegalArgumentException("Cipher Suite not enabled: " + gotAllEnabled))
|
||||
engine.getSupportedProtocols.contains(settings.SSLProtocol.get) || (throw new IllegalArgumentException("Protocol not supported: " + settings.SSLProtocol.get))
|
||||
|
||||
(cipher, true, config)
|
||||
} catch {
|
||||
case (_: IllegalArgumentException) | (_: NoSuchAlgorithmException) ⇒ (cipher, false, AkkaSpec.testConf) // Cannot match against the message since the message might be localized :S
|
||||
}
|
||||
}
|
||||
|
||||
import Configuration.getCipherConfig
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978SHA1PRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("SHA1PRNG", "TLS_RSA_WITH_AES_128_CBC_SHA"))
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978AES128CounterRNGFastSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES128CounterRNGFast", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
|
||||
|
||||
/**
|
||||
* Both of the <quote>Secure</quote> variants require access to the Internet to access random.org.
|
||||
*/
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978AES128CounterRNGSecureSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES128CounterRNGSecure", "TLS_RSA_WITH_AES_128_CBC_SHA"))
|
||||
|
||||
/**
|
||||
* Both of the <quote>Secure</quote> variants require access to the Internet to access random.org.
|
||||
*/
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978AES256CounterRNGSecureSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES256CounterRNGSecure", "TLS_RSA_WITH_AES_256_CBC_SHA"))
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978DefaultRNGSecureSpec extends Ticket1978CommunicationSpec(getCipherConfig("", "TLS_RSA_WITH_AES_128_CBC_SHA"))
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978NonExistingRNGSecureSpec extends Ticket1978CommunicationSpec(("NonExistingRNG", false, AkkaSpec.testConf))
|
||||
|
||||
abstract class Ticket1978CommunicationSpec(val cipherEnabledconfig: (String, Boolean, Config)) extends AkkaSpec(cipherEnabledconfig._3) with ImplicitSender {
|
||||
|
||||
implicit val timeout: Timeout = Timeout(5 seconds)
|
||||
|
||||
import RemoteCommunicationSpec._
|
||||
|
||||
val other = ActorSystem("remote-sys", ConfigFactory.parseString("akka.remote.netty.port=12346").withFallback(system.settings.config))
|
||||
|
||||
override def atTermination() {
|
||||
other.shutdown()
|
||||
other.awaitTermination()
|
||||
}
|
||||
|
||||
"SSL Remoting" must {
|
||||
if (cipherEnabledconfig._2) {
|
||||
val remote = other.actorOf(Props(new Actor { def receive = { case "ping" ⇒ sender ! (("pong", sender)) } }), "echo")
|
||||
|
||||
val here = system.actorFor("akka://remote-sys@localhost:12346/user/echo")
|
||||
|
||||
"support remote look-ups" in {
|
||||
here ! "ping"
|
||||
expectMsgPF(timeout.duration) {
|
||||
case ("pong", s: AnyRef) if s eq testActor ⇒ true
|
||||
}
|
||||
}
|
||||
|
||||
"send error message for wrong address" ignore {
|
||||
within(timeout.duration) {
|
||||
EventFilter.error(start = "dropping", occurrences = 1).intercept {
|
||||
system.actorFor("akka://remotesys@localhost:12346/user/echo") ! "ping"
|
||||
}(other)
|
||||
}
|
||||
}
|
||||
|
||||
"support ask" in {
|
||||
Await.result(here ? "ping", timeout.duration) match {
|
||||
case ("pong", s: akka.pattern.PromiseActorRef) ⇒ // good
|
||||
case m ⇒ fail(m + " was not (pong, AskActorRef)")
|
||||
}
|
||||
}
|
||||
|
||||
"send dead letters on remote if actor does not exist" in {
|
||||
within(timeout.duration) {
|
||||
EventFilter.warning(pattern = "dead.*buh", occurrences = 1).intercept {
|
||||
system.actorFor("akka://remote-sys@localhost:12346/does/not/exist") ! "buh"
|
||||
}(other)
|
||||
}
|
||||
}
|
||||
|
||||
"create and supervise children on remote node" in {
|
||||
within(timeout.duration) {
|
||||
val r = system.actorOf(Props[Echo], "blub")
|
||||
r.path.toString must be === "akka://remote-sys@localhost:12346/remote/Ticket1978CommunicationSpec@localhost:12345/user/blub"
|
||||
r ! 42
|
||||
expectMsg(42)
|
||||
EventFilter[Exception]("crash", occurrences = 1).intercept {
|
||||
r ! new Exception("crash")
|
||||
}(other)
|
||||
expectMsg("preRestart")
|
||||
r ! 42
|
||||
expectMsg(42)
|
||||
}
|
||||
}
|
||||
|
||||
"look-up actors across node boundaries" in {
|
||||
within(timeout.duration) {
|
||||
val l = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case (p: Props, n: String) ⇒ sender ! context.actorOf(p, n)
|
||||
case s: String ⇒ sender ! context.actorFor(s)
|
||||
}
|
||||
}), "looker")
|
||||
l ! (Props[Echo], "child")
|
||||
val r = expectMsgType[ActorRef]
|
||||
r ! (Props[Echo], "grandchild")
|
||||
val remref = expectMsgType[ActorRef]
|
||||
remref.isInstanceOf[LocalActorRef] must be(true)
|
||||
val myref = system.actorFor(system / "looker" / "child" / "grandchild")
|
||||
myref.isInstanceOf[RemoteActorRef] must be(true)
|
||||
myref ! 43
|
||||
expectMsg(43)
|
||||
lastSender must be theSameInstanceAs remref
|
||||
r.asInstanceOf[RemoteActorRef].getParent must be(l)
|
||||
system.actorFor("/user/looker/child") must be theSameInstanceAs r
|
||||
Await.result(l ? "child/..", timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
|
||||
Await.result(system.actorFor(system / "looker" / "child") ? "..", timeout.duration).asInstanceOf[AnyRef] must be theSameInstanceAs l
|
||||
}
|
||||
}
|
||||
|
||||
"not fail ask across node boundaries" in {
|
||||
val f = for (_ ← 1 to 1000) yield here ? "ping" mapTo manifest[(String, ActorRef)]
|
||||
Await.result(Future.sequence(f), timeout.duration).map(_._1).toSet must be(Set("pong"))
|
||||
}
|
||||
} else {
|
||||
"not be run when the cipher is not supported by the platform this test is currently being executed on" ignore {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,48 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.testkit._
|
||||
import akka.actor._
|
||||
import com.typesafe.config._
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.util.duration._
|
||||
import akka.util.Duration
|
||||
import akka.remote.netty.NettyRemoteTransport
|
||||
import java.util.ArrayList
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Ticket1978ConfigSpec extends AkkaSpec("""
|
||||
akka {
|
||||
actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
remote.netty {
|
||||
hostname = localhost
|
||||
port = 12345
|
||||
}
|
||||
actor.deployment {
|
||||
/blub.remote = "akka://remote-sys@localhost:12346"
|
||||
/looker/child.remote = "akka://remote-sys@localhost:12346"
|
||||
/looker/child/grandchild.remote = "akka://RemoteCommunicationSpec@localhost:12345"
|
||||
}
|
||||
}
|
||||
""") with ImplicitSender with DefaultTimeout {
|
||||
|
||||
"SSL Remoting" must {
|
||||
"be able to parse these extra Netty config elements" in {
|
||||
val settings =
|
||||
system.asInstanceOf[ExtendedActorSystem]
|
||||
.provider.asInstanceOf[RemoteActorRefProvider]
|
||||
.transport.asInstanceOf[NettyRemoteTransport]
|
||||
.settings
|
||||
import settings._
|
||||
|
||||
EnableSSL must be(false)
|
||||
SSLKeyStore must be(Some("keystore"))
|
||||
SSLKeyStorePassword must be(Some("changeme"))
|
||||
SSLTrustStore must be(Some("truststore"))
|
||||
SSLTrustStorePassword must be(Some("changeme"))
|
||||
SSLProtocol must be(Some("TLSv1"))
|
||||
SSLEnabledAlgorithms must be(Set("TLS_RSA_WITH_AES_128_CBC_SHA"))
|
||||
SSLRandomSource must be(None)
|
||||
SSLRandomNumberGenerator must be(None)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -75,7 +75,9 @@ object AkkaKernelPlugin extends Plugin {
|
|||
|
||||
copyFiles(libFiles(cp, conf.libFilter), distLibPath)
|
||||
copyFiles(conf.additionalLibs, distLibPath)
|
||||
for (subTarget ← subProjectDependencies.map(_.target)) {
|
||||
for (subProjectDependency ← subProjectDependencies) {
|
||||
val subTarget = subProjectDependency.target
|
||||
EvaluateTask(buildStruct, packageBin in Compile, st, subProjectDependency.projectRef)
|
||||
copyJars(subTarget, distLibPath)
|
||||
}
|
||||
log.info("Distribution created.")
|
||||
|
|
@ -220,10 +222,10 @@ object AkkaKernelPlugin extends Plugin {
|
|||
}.toList
|
||||
|
||||
val target = setting(Keys.crossTarget, "Missing crossTarget directory")
|
||||
SubProjectInfo(project.id, target, subProjects)
|
||||
SubProjectInfo(projectRef, target, subProjects)
|
||||
}
|
||||
|
||||
private case class SubProjectInfo(id: String, target: File, subProjects: Seq[SubProjectInfo]) {
|
||||
private case class SubProjectInfo(projectRef: ProjectRef, target: File, subProjects: Seq[SubProjectInfo]) {
|
||||
|
||||
def recursiveSubProjects: Set[SubProjectInfo] = {
|
||||
val flatSubProjects = for {
|
||||
|
|
|
|||
|
|
@ -5,9 +5,7 @@
|
|||
package akka.testkit
|
||||
|
||||
import akka.actor._
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import scala.collection.immutable.Stack
|
||||
import akka.dispatch._
|
||||
import akka.pattern.ask
|
||||
|
||||
|
|
|
|||
|
|
@ -158,7 +158,13 @@ trait TestKitBase {
|
|||
* block or missing that it returns the properly dilated default for this
|
||||
* case from settings (key "akka.test.single-expect-default").
|
||||
*/
|
||||
def remaining: Duration = if (end == Duration.Undefined) testKitSettings.SingleExpectDefaultTimeout.dilated else end - now
|
||||
def remaining: Duration = remainingOr(testKitSettings.SingleExpectDefaultTimeout.dilated)
|
||||
|
||||
/**
|
||||
* Obtain time remaining for execution of the innermost enclosing `within`
|
||||
* block or missing that it returns the given duration.
|
||||
*/
|
||||
def remainingOr(duration: Duration): Duration = if (end == Duration.Undefined) duration else end - now
|
||||
|
||||
/**
|
||||
* Query queue status.
|
||||
|
|
@ -486,7 +492,8 @@ trait TestKitBase {
|
|||
|
||||
@tailrec
|
||||
def doit(acc: List[T], count: Int): List[T] = {
|
||||
if (count >= messages) return acc.reverse
|
||||
if (count >= messages) acc.reverse
|
||||
else {
|
||||
receiveOne((stop - now) min idle)
|
||||
lastMessage match {
|
||||
case NullMessage ⇒
|
||||
|
|
@ -501,6 +508,7 @@ trait TestKitBase {
|
|||
acc.reverse
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val ret = doit(Nil, 0)
|
||||
lastWasNoMsg = true
|
||||
|
|
@ -605,12 +613,6 @@ object TestKit {
|
|||
/**
|
||||
* Await until the given condition evaluates to `true` or the timeout
|
||||
* expires, whichever comes first.
|
||||
*
|
||||
* If no timeout is given, take it from the innermost enclosing `within`
|
||||
* block.
|
||||
*
|
||||
* Note that the timeout is scaled using Duration.dilated, which uses the
|
||||
* configuration entry "akka.test.timefactor"
|
||||
*/
|
||||
def awaitCond(p: ⇒ Boolean, max: Duration, interval: Duration = 100.millis, noThrow: Boolean = false): Boolean = {
|
||||
val stop = now + max
|
||||
|
|
|
|||
|
|
@ -57,7 +57,7 @@ public class UntypedCoordinatedIncrementTest {
|
|||
Timeout timeout = new Timeout(timeoutSeconds, TimeUnit.SECONDS);
|
||||
|
||||
@Before
|
||||
public void initialise() {
|
||||
public void initialize() {
|
||||
counters = new ArrayList<ActorRef>();
|
||||
for (int i = 1; i <= numCounters; i++) {
|
||||
final String name = "counter" + i;
|
||||
|
|
|
|||
|
|
@ -58,7 +58,7 @@ public class UntypedTransactorTest {
|
|||
Timeout timeout = new Timeout(timeoutSeconds, TimeUnit.SECONDS);
|
||||
|
||||
@Before
|
||||
public void initialise() {
|
||||
public void initialize() {
|
||||
counters = new ArrayList<ActorRef>();
|
||||
for (int i = 1; i <= numCounters; i++) {
|
||||
final String name = "counter" + i;
|
||||
|
|
|
|||
|
|
@ -9,14 +9,17 @@ import akka.actor._
|
|||
import akka.dispatch.{ Promise, Future }
|
||||
import akka.event.Logging
|
||||
import annotation.tailrec
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.TimeUnit
|
||||
import collection.mutable.ListBuffer
|
||||
import akka.util.{ NonFatal, Duration }
|
||||
|
||||
private[zeromq] object ConcurrentSocketActor {
|
||||
private sealed trait PollMsg
|
||||
private case object Poll extends PollMsg
|
||||
private case object PollCareful extends PollMsg
|
||||
|
||||
private case object Flush
|
||||
|
||||
private class NoSocketHandleException() extends Exception("Couldn't create a zeromq socket.")
|
||||
|
||||
private val DefaultContext = Context()
|
||||
|
|
@ -32,19 +35,28 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A
|
|||
import SocketType.{ ZMQSocketType ⇒ ST }
|
||||
params.collectFirst { case t: ST ⇒ t }.getOrElse(throw new IllegalArgumentException("A socket type is required"))
|
||||
}
|
||||
|
||||
private val socket: Socket = zmqContext.socket(socketType)
|
||||
private val poller: Poller = zmqContext.poller
|
||||
private val log = Logging(context.system, this)
|
||||
|
||||
private val pendingSends = new ListBuffer[Seq[Frame]]
|
||||
|
||||
def receive = {
|
||||
case m: PollMsg ⇒ doPoll(m)
|
||||
case ZMQMessage(frames) ⇒ sendMessage(frames)
|
||||
case ZMQMessage(frames) ⇒ handleRequest(Send(frames))
|
||||
case r: Request ⇒ handleRequest(r)
|
||||
case Flush ⇒ flush()
|
||||
case Terminated(_) ⇒ context stop self
|
||||
}
|
||||
|
||||
private def handleRequest(msg: Request): Unit = msg match {
|
||||
case Send(frames) ⇒ sendMessage(frames)
|
||||
case Send(frames) ⇒
|
||||
if (frames.nonEmpty) {
|
||||
val flushNow = pendingSends.isEmpty
|
||||
pendingSends.append(frames)
|
||||
if (flushNow) flush()
|
||||
}
|
||||
case opt: SocketOption ⇒ handleSocketOption(opt)
|
||||
case q: SocketOptionQuery ⇒ handleSocketOptionQuery(q)
|
||||
}
|
||||
|
|
@ -117,48 +129,46 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A
|
|||
}
|
||||
}
|
||||
|
||||
private def setupConnection() {
|
||||
private def setupConnection(): Unit = {
|
||||
params filter (_.isInstanceOf[SocketConnectOption]) foreach { self ! _ }
|
||||
params filter (_.isInstanceOf[PubSubOption]) foreach { self ! _ }
|
||||
}
|
||||
|
||||
private def deserializerFromParams = {
|
||||
private def deserializerFromParams: Deserializer =
|
||||
params collectFirst { case d: Deserializer ⇒ d } getOrElse new ZMQMessageDeserializer
|
||||
}
|
||||
|
||||
private def setupSocket() = {
|
||||
params foreach {
|
||||
private def setupSocket() = params foreach {
|
||||
case _: SocketConnectOption | _: PubSubOption | _: SocketMeta ⇒ // ignore, handled differently
|
||||
case m ⇒ self ! m
|
||||
}
|
||||
}
|
||||
|
||||
override def preRestart(reason: Throwable, message: Option[Any]) {
|
||||
context.children foreach context.stop //Do not call postStop
|
||||
}
|
||||
override def preRestart(reason: Throwable, message: Option[Any]): Unit = context.children foreach context.stop //Do not call postStop
|
||||
|
||||
override def postRestart(reason: Throwable) {} //Do nothing
|
||||
override def postRestart(reason: Throwable): Unit = () // Do nothing
|
||||
|
||||
override def postStop {
|
||||
try {
|
||||
override def postStop: Unit = try {
|
||||
if (socket != null) {
|
||||
poller.unregister(socket)
|
||||
socket.close
|
||||
}
|
||||
} finally {
|
||||
notifyListener(Closed)
|
||||
} finally notifyListener(Closed)
|
||||
|
||||
@tailrec private def flushMessage(i: Seq[Frame]): Boolean =
|
||||
if (i.isEmpty)
|
||||
true
|
||||
else {
|
||||
val head = i.head
|
||||
val tail = i.tail
|
||||
if (socket.send(head.payload.toArray, if (tail.nonEmpty) JZMQ.SNDMORE else 0)) flushMessage(tail)
|
||||
else {
|
||||
pendingSends.prepend(i) // Reenqueue the rest of the message so the next flush takes care of it
|
||||
self ! Flush
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
private def sendMessage(frames: Seq[Frame]) {
|
||||
def sendBytes(bytes: Seq[Byte], flags: Int) = socket.send(bytes.toArray, flags)
|
||||
val iter = frames.iterator
|
||||
while (iter.hasNext) {
|
||||
val payload = iter.next.payload
|
||||
val flags = if (iter.hasNext) JZMQ.SNDMORE else 0
|
||||
sendBytes(payload, flags)
|
||||
}
|
||||
}
|
||||
@tailrec private def flush(): Unit =
|
||||
if (pendingSends.nonEmpty && flushMessage(pendingSends.remove(0))) flush() // Flush while things are going well
|
||||
|
||||
// this is a “PollMsg=>Unit” which either polls or schedules Poll, depending on the sign of the timeout
|
||||
private val doPollTimeout = {
|
||||
|
|
|
|||
|
|
@ -139,8 +139,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension {
|
|||
*/
|
||||
def newSocket(socketParameters: SocketOption*): ActorRef = {
|
||||
implicit val timeout = NewSocketTimeout
|
||||
val req = (zeromqGuardian ? newSocketProps(socketParameters: _*)).mapTo[ActorRef]
|
||||
Await.result(req, timeout.duration)
|
||||
Await.result((zeromqGuardian ? newSocketProps(socketParameters: _*)).mapTo[ActorRef], timeout.duration)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -248,9 +247,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension {
|
|||
case _ ⇒ false
|
||||
}
|
||||
|
||||
def receive = {
|
||||
case p: Props ⇒ sender ! context.actorOf(p)
|
||||
}
|
||||
def receive = { case p: Props ⇒ sender ! context.actorOf(p) }
|
||||
}), "zeromq")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -449,7 +449,7 @@ object Dependencies {
|
|||
|
||||
val actorTests = Seq(Test.junit, Test.scalatest, Test.commonsMath, Test.mockito, Test.scalacheck, protobuf)
|
||||
|
||||
val remote = Seq(netty, protobuf, Test.junit, Test.scalatest)
|
||||
val remote = Seq(netty, protobuf, uncommonsMath, Test.junit, Test.scalatest)
|
||||
|
||||
val cluster = Seq(Test.junit, Test.scalatest)
|
||||
|
||||
|
|
@ -483,6 +483,8 @@ object Dependency {
|
|||
val scalaStm = "org.scala-tools" %% "scala-stm" % "0.5" // Modified BSD (Scala)
|
||||
val slf4jApi = "org.slf4j" % "slf4j-api" % "1.6.4" // MIT
|
||||
val zeroMQ = "org.zeromq" %% "zeromq-scala-binding" % "0.0.6" // ApacheV2
|
||||
val uncommonsMath = "org.uncommons.maths" % "uncommons-maths" % "1.2.2a" // ApacheV2
|
||||
|
||||
|
||||
// Test
|
||||
|
||||
|
|
@ -492,8 +494,8 @@ object Dependency {
|
|||
val junit = "junit" % "junit" % "4.10" % "test" // Common Public License 1.0
|
||||
val logback = "ch.qos.logback" % "logback-classic" % "1.0.4" % "test" // EPL 1.0 / LGPL 2.1
|
||||
val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" // MIT
|
||||
val scalatest = "org.scalatest" %% "scalatest" % "1.8-SNAPSHOT" % "test" // ApacheV2
|
||||
val scalacheck = "org.scalacheck" % "scalacheck_2.10.0-M3" % "1.10-SNAPSHOT" % "test" // New BSD
|
||||
val scalatest = "org.scalatest" %% "scalatest" % "1.9-2.10.0-M4-B1" % "test" // ApacheV2
|
||||
val scalacheck = "org.scalacheck" %% "scalacheck" % "1.10.0-b1" % "test" // New BSD
|
||||
val specs2 = "org.specs2" %% "specs2" % "1.11" % "test" // Modified BSD / ApacheV2
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue