Mergin in the latest master

This commit is contained in:
Viktor Klang 2012-06-20 11:28:40 +02:00
commit 1711934043
83 changed files with 2846 additions and 1027 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

Binary file not shown.

Binary file not shown.

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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