Merge branch 'master' into wip-2053d-actorbased-remote-drewhk

Conflicts:
	akka-docs/rst/java/code/docs/serialization/SerializationDocTestBase.java
	akka-docs/rst/scala/code/docs/serialization/SerializationDocSpec.scala
	akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala
	akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
This commit is contained in:
Endre Sándor Varga 2012-12-18 15:15:01 +01:00
commit 55be17419e
213 changed files with 5004 additions and 1940 deletions

View file

@ -1,8 +1,11 @@
package akka.japi;
import akka.actor.ExtendedActorSystem;
import akka.event.LoggingAdapter;
import akka.event.NoLogging;
import akka.serialization.JavaSerializer;
import org.junit.Test;
import java.util.concurrent.Callable;
import static org.junit.Assert.*;
@ -54,4 +57,13 @@ public class JavaAPITestBase {
LoggingAdapter a = NoLogging.getInstance();
assertNotNull(a);
}
@Test
public void mustBeAbleToUseCurrentSystem() {
assertNull(JavaSerializer.currentSystem().withValue(null, new Callable<ExtendedActorSystem>() {
public ExtendedActorSystem call() {
return JavaSerializer.currentSystem().value();
}
}));
}
}

View file

@ -4,7 +4,6 @@
package akka.actor
import language.postfixOps
import akka.testkit._
import org.scalatest.junit.JUnitSuite
import com.typesafe.config.ConfigFactory
@ -15,6 +14,10 @@ import akka.util.Timeout
import akka.japi.Util.immutableSeq
import scala.concurrent.Future
import akka.pattern.ask
import akka.dispatch._
import com.typesafe.config.Config
import java.util.concurrent.{ LinkedBlockingQueue, BlockingQueue, TimeUnit }
import akka.util.Switch
class JavaExtensionSpec extends JavaExtension with JUnitSuite
@ -68,10 +71,57 @@ object ActorSystemSpec {
}
}
case class FastActor(latch: TestLatch, testActor: ActorRef) extends Actor {
val ref1 = context.actorOf(Props.empty)
val ref2 = context.actorFor(ref1.path.toString)
testActor ! ref2.getClass
latch.countDown()
def receive = {
case _
}
}
class SlowDispatcher(_config: Config, _prerequisites: DispatcherPrerequisites) extends MessageDispatcherConfigurator(_config, _prerequisites) {
private val instance = new Dispatcher(
prerequisites,
config.getString("id"),
config.getInt("throughput"),
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType,
configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) {
val doneIt = new Switch
override protected[akka] def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = {
val ret = super.registerForExecution(mbox, hasMessageHint, hasSystemMessageHint)
doneIt.switchOn {
TestKit.awaitCond(mbox.actor.actor != null, 1.second)
mbox.actor.actor match {
case FastActor(latch, _) Await.ready(latch, 1.second)
}
}
ret
}
}
/**
* Returns the same dispatcher instance for each invocation
*/
override def dispatcher(): MessageDispatcher = instance
}
val config = s"""
akka.extensions = ["akka.actor.TestExtension"]
slow {
type="${classOf[SlowDispatcher].getName}"
}"""
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension"]""") with ImplicitSender {
class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSender {
import ActorSystemSpec.FastActor
"An ActorSystem" must {
@ -165,6 +215,11 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
Await.result(Future.sequence(waves), timeout.duration + 5.seconds) must be === Seq("done", "done", "done")
}
"find actors that just have been created" in {
system.actorOf(Props(new FastActor(TestLatch(), testActor)).withDispatcher("slow"))
expectMsgType[Class[_]] must be(classOf[LocalActorRef])
}
"reliable deny creation of actors while shutting down" in {
val system = ActorSystem()
import system.dispatcher

View file

@ -13,6 +13,9 @@ import akka.event._
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import akka.util.Timeout
import org.scalatest.matchers.Matcher
import org.scalatest.matchers.HavePropertyMatcher
import org.scalatest.matchers.HavePropertyMatchResult
object FSMActorSpec {
val timeout = Timeout(2 seconds)
@ -199,6 +202,45 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
expectMsg(1 second, fsm.StopEvent(FSM.Shutdown, 1, null))
}
"cancel all timers when terminated" in {
val timerNames = List("timer-1", "timer-2", "timer-3")
// Lazy so fsmref can refer to checkTimersActive
lazy val fsmref = TestFSMRef(new Actor with FSM[String, Null] {
startWith("not-started", null)
when("not-started") {
case Event("start", _) goto("started") replying "starting"
}
when("started", stateTimeout = 10 seconds) {
case Event("stop", _) stop()
}
onTransition {
case "not-started" -> "started"
for (timerName timerNames) setTimer(timerName, (), 10 seconds, false)
}
onTermination {
case _ {
checkTimersActive(false)
testActor ! "stopped"
}
}
})
def checkTimersActive(active: Boolean) {
for (timer timerNames) fsmref.isTimerActive(timer) must be(active)
fsmref.isStateTimerActive must be(active)
}
checkTimersActive(false)
fsmref ! "start"
expectMsg(1 second, "starting")
checkTimersActive(true)
fsmref ! "stop"
expectMsg(1 second, "stopped")
}
"log events and transitions if asked to do so" in {
import scala.collection.JavaConverters._
val config = ConfigFactory.parseMap(Map("akka.loglevel" -> "DEBUG",

View file

@ -11,7 +11,7 @@ import akka.pattern.ask
import java.util.concurrent.atomic.AtomicInteger
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {
class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout with ImplicitSender {
private val cancellables = new ConcurrentLinkedQueue[Cancellable]()
import system.dispatcher
@ -33,39 +33,47 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
"schedule more than once" in {
case object Tick
val countDownLatch = new CountDownLatch(3)
val tickActor = system.actorOf(Props(new Actor {
def receive = { case Tick countDownLatch.countDown() }
case object Tock
val tickActor, tickActor2 = system.actorOf(Props(new Actor {
var ticks = 0
def receive = {
case Tick
if (ticks < 3) {
sender ! Tock
ticks += 1
}
}
}))
// run every 50 milliseconds
collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds, tickActor, Tick))
// after max 1 second it should be executed at least the 3 times already
assert(countDownLatch.await(2, TimeUnit.SECONDS))
expectMsg(Tock)
expectMsg(Tock)
expectMsg(Tock)
expectNoMsg(500 millis)
val countDownLatch2 = new CountDownLatch(3)
collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds)(countDownLatch2.countDown()))
collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds)(tickActor2 ! Tick))
// after max 1 second it should be executed at least the 3 times already
assert(countDownLatch2.await(2, TimeUnit.SECONDS))
expectMsg(Tock)
expectMsg(Tock)
expectMsg(Tock)
expectNoMsg(500 millis)
}
"stop continuous scheduling if the receiving actor has been terminated" taggedAs TimingTest in {
val actor = system.actorOf(Props(new Actor {
def receive = {
case x testActor ! x
}
}))
val actor = system.actorOf(Props(new Actor { def receive = { case x sender ! x } }))
// run immediately and then every 100 milliseconds
collectCancellable(system.scheduler.schedule(0 milliseconds, 100 milliseconds, actor, "msg"))
expectMsg("msg")
// stop the actor and, hence, the continuous messaging from happening
actor ! PoisonPill
system stop actor
expectNoMsg(1 second)
expectNoMsg(500 millis)
}
"schedule once" in {
@ -93,19 +101,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
* ticket #372
*/
"be cancellable" in {
object Ping
val ticks = new CountDownLatch(1)
for (_ 1 to 10) system.scheduler.scheduleOnce(1 second, testActor, "fail").cancel()
val actor = system.actorOf(Props(new Actor {
def receive = { case Ping ticks.countDown() }
}))
(1 to 10).foreach { i
val timeout = collectCancellable(system.scheduler.scheduleOnce(1 second, actor, Ping))
timeout.cancel()
}
assert(ticks.await(3, TimeUnit.SECONDS) == false) //No counting down should've been made
expectNoMsg(2 seconds)
}
"be cancellable during initial delay" taggedAs TimingTest in {
@ -200,31 +198,24 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
case object Msg
val actor = system.actorOf(Props(new Actor {
def receive = {
case Msg ticks.countDown()
}
def receive = { case Msg ticks.countDown() }
}))
val startTime = System.nanoTime()
val cancellable = system.scheduler.schedule(1 second, 300 milliseconds, actor, Msg)
collectCancellable(system.scheduler.schedule(1 second, 300 milliseconds, actor, Msg))
Await.ready(ticks, 3 seconds)
val elapsedTimeMs = (System.nanoTime() - startTime) / 1000000
assert(elapsedTimeMs > 1600)
assert(elapsedTimeMs < 2000) // the precision is not ms exact
cancellable.cancel()
(System.nanoTime() - startTime).nanos.toMillis must be(1800L plusOrMinus 199)
}
"adjust for scheduler inaccuracy" taggedAs TimingTest in {
val startTime = System.nanoTime
val n = 33
val latch = new TestLatch(n)
system.scheduler.schedule(150.millis, 150.millis) {
latch.countDown()
}
system.scheduler.schedule(150.millis, 150.millis) { latch.countDown() }
Await.ready(latch, 6.seconds)
val rate = n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis
rate must be(6.66 plusOrMinus (0.4))
// Rate
n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis must be(6.66 plusOrMinus 0.4)
}
"not be affected by long running task" taggedAs TimingTest in {
@ -236,8 +227,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
latch.countDown()
}
Await.ready(latch, 6.seconds)
val rate = n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis
rate must be(4.4 plusOrMinus (0.3))
// Rate
n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis must be(4.4 plusOrMinus 0.3)
}
}
}

View file

@ -94,6 +94,14 @@ object TypedActorSpec {
@throws(classOf[TimeoutException])
def joptionPigdog(delay: Long): JOption[String]
def nullFuture(): Future[Any] = null
def nullJOption(): JOption[Any] = null
def nullOption(): Option[Any] = null
def nullReturn(): Any = null
def incr()
@throws(classOf[TimeoutException])
@ -283,6 +291,14 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
mustStop(t)
}
"be able to call null returning methods" in {
val t = newFooBar
t.nullJOption() must be === JOption.none
t.nullOption() must be === None
t.nullReturn() must be === null
Await.result(t.nullFuture(), remaining) must be === null
}
"be able to call Future-returning methods non-blockingly" in {
val t = newFooBar
val f = t.futurePigdog(200)

View file

@ -51,7 +51,12 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
def testOrdering(dispatcherKey: String) {
val msgs = (1 to 100) toList
// It's important that the actor under test is not a top level actor
// with RepointableActorRef, since messages might be queued in
// UnstartedCell and the sent to the PriorityQueue and consumed immediately
// without the ordering taking place.
val actor = system.actorOf(Props(new Actor {
context.actorOf(Props(new Actor {
val acc = scala.collection.mutable.ListBuffer[Int]()
@ -65,6 +70,10 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
}
}).withDispatcher(dispatcherKey))
def receive = Actor.emptyBehavior
}))
expectMsgType[List[_]] must be === msgs
}

View file

@ -8,6 +8,7 @@ import language.postfixOps
import akka.testkit.AkkaSpec
import akka.actor.{ Props, Actor }
import java.util.concurrent.TimeoutException
import scala.concurrent.{ Future, Promise, Await }
import scala.concurrent.duration._
@ -39,10 +40,10 @@ class PatternSpec extends AkkaSpec {
Await.ready(gracefulStop(target, 1 millis), 1 second)
}
"complete Future with AskTimeoutException when actor not terminated within timeout" in {
"complete Future with TimeoutException when actor not terminated within timeout" in {
val target = system.actorOf(Props[TargetActor])
target ! Work(250 millis)
intercept[AskTimeoutException] { Await.result(gracefulStop(target, 10 millis), 200 millis) }
intercept[TimeoutException] { Await.result(gracefulStop(target, 10 millis), 200 millis) }
}
}

View file

@ -173,19 +173,20 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
routeeSize(router) must be(resizer.upperBound)
}
"backoff" in {
"backoff" in within(10 seconds) {
val resizer = DefaultResizer(
lowerBound = 1,
upperBound = 5,
rampupRate = 1.0,
backoffRate = 1.0,
backoffThreshold = 0.20,
backoffThreshold = 0.40,
pressureThreshold = 1,
messagesPerResize = 1)
val router = system.actorOf(Props(new Actor {
def receive = {
case n: Int if n <= 0 // done
case n: Int Thread.sleep((n millis).dilated.toMillis)
}
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
@ -202,12 +203,11 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
Thread.sleep((300 millis).dilated.toMillis)
// let it cool down
for (m 0 to 5) {
router ! 1
Thread.sleep((500 millis).dilated.toMillis)
}
awaitCond({
router ! 0 // trigger resize
routeeSize(router) < z
}, interval = 500.millis.dilated)
awaitCond(Try(routeeSize(router) < (z)).getOrElse(false))
}
}

View file

@ -118,6 +118,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
val names = 1 to 20 map { "routee" + _ } toList
actor ! TestRun("test", names, actors)
1 to actors foreach { _
val routees = expectMsgType[RouterRoutees].routees
routees.map(_.path.name) must be === names

View file

@ -8,10 +8,12 @@ import akka.util.Unsafe;
final class AbstractActorRef {
final static long cellOffset;
final static long lookupOffset;
static {
try {
cellOffset = Unsafe.instance.objectFieldOffset(RepointableActorRef.class.getDeclaredField("_cellDoNotCallMeDirectly"));
lookupOffset = Unsafe.instance.objectFieldOffset(RepointableActorRef.class.getDeclaredField("_lookupDoNotCallMeDirectly"));
} catch(Throwable t){
throw new ExceptionInInitializerError(t);
}

View file

@ -24,6 +24,7 @@ import java.util.concurrent.ThreadFactory;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import akka.util.Helpers;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import akka.event.LoggingAdapter;
@ -91,7 +92,6 @@ public class HashedWheelTimer implements Timer {
final ReusableIterator<HashedWheelTimeout>[] iterators;
final int mask;
final ReadWriteLock lock = new ReentrantReadWriteLock();
final boolean isWindows = System.getProperty("os.name", "").toLowerCase().indexOf("win") >= 0;
volatile int wheelCursor;
private LoggingAdapter logger;
@ -396,7 +396,7 @@ public class HashedWheelTimer implements Timer {
// the JVM if it runs on windows.
//
// See https://github.com/netty/netty/issues/356
if (isWindows) {
if (Helpers.isWindows()) {
sleepTimeMs = (sleepTimeMs / 10) * 10;
}

View file

@ -82,8 +82,9 @@ case class Terminated private[akka] (@BeanProperty actor: ActorRef)(
* INTERNAL API
*
* Used for remote death watch. Failure detector publish this to the
* `eventStream` when a remote node is detected to be unreachable.
* The watcher ([[akka.actor.DeathWatch]]) subscribes to the `eventStream`
* `eventStream` when a remote node is detected to be unreachable and/or decided to
* be removed.
* The watcher ([[akka.actor.dungeon.DeathWatch]]) subscribes to the `eventStream`
* and translates this event to [[akka.actor.Terminated]], which is sent itself.
*/
@SerialVersionUID(1L)
@ -179,8 +180,8 @@ object ActorInitializationException {
*
* @param actor is the actor whose preRestart() hook failed
* @param cause is the exception thrown by that actor within preRestart()
* @param origCause is the exception which caused the restart in the first place
* @param msg is the message which was optionally passed into preRestart()
* @param originalCause is the exception which caused the restart in the first place
* @param messageOption is the message which was optionally passed into preRestart()
*/
@SerialVersionUID(1L)
case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable, messageOption: Option[Any])
@ -196,7 +197,7 @@ case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable,
*
* @param actor is the actor whose constructor or postRestart() hook failed
* @param cause is the exception thrown by that actor within preRestart()
* @param origCause is the exception which caused the restart in the first place
* @param originalCause is the exception which caused the restart in the first place
*/
@SerialVersionUID(1L)
case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable)

View file

@ -208,6 +208,11 @@ private[akka] trait Cell {
* The system internals where this Cell lives.
*/
def systemImpl: ActorSystemImpl
/**
* Start the cell: enqueued message must not be processed before this has
* been called. The usual action is to attach the mailbox to a dispatcher.
*/
def start(): this.type
/**
* Recursively suspend this actor and all its children. Must not throw exceptions.
*/
@ -362,7 +367,7 @@ private[akka] class ActorCell(
case w: WaitingForChildren w.enqueue(message)
}
case Terminate() terminate()
case Supervise(child, uid) supervise(child, uid)
case Supervise(child, async, uid) supervise(child, async, uid)
case ChildTerminated(child) todo = handleChildTerminated(child)
case NoMessage // only here to suppress warning
}
@ -492,20 +497,20 @@ private[akka] class ActorCell(
}
}
private def supervise(child: ActorRef, uid: Int): Unit = if (!isTerminating) {
private def supervise(child: ActorRef, async: Boolean, uid: Int): Unit = if (!isTerminating) {
// Supervise is the first thing we get from a new child, so store away the UID for later use in handleFailure()
initChild(child) match {
case Some(crs)
crs.uid = uid
handleSupervise(child)
handleSupervise(child, async)
if (system.settings.DebugLifecycle) publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
case None publish(Error(self.path.toString, clazz(actor), "received Supervise from unregistered child " + child + ", this will not end well"))
}
}
// future extension point
protected def handleSupervise(child: ActorRef): Unit = child match {
case r: RepointableActorRef r.activate()
protected def handleSupervise(child: ActorRef, async: Boolean): Unit = child match {
case r: RepointableActorRef if async r.point()
case _
}

View file

@ -42,8 +42,7 @@ trait ActorRefProvider {
def deadLetters: ActorRef
/**
* The root path for all actors within this actor system, including remote
* address if enabled.
* The root path for all actors within this actor system, not including any remote address information.
*/
def rootPath: ActorPath
@ -146,6 +145,11 @@ trait ActorRefProvider {
* attempt is made to verify actual reachability).
*/
def getExternalAddressFor(addr: Address): Option[Address]
/**
* Obtain the external address of the default transport.
*/
def getDefaultAddress: Address
}
/**
@ -317,6 +321,10 @@ private[akka] object SystemGuardian {
/**
* Local ActorRef provider.
*
* INTERNAL API!
*
* Depending on this class is not supported, only the [[ActorRefProvider]] interface is supported.
*/
class LocalActorRefProvider(
_systemName: String,
@ -381,7 +389,7 @@ class LocalActorRefProvider(
override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff {
message match {
case Supervise(_, _) // TODO register child in some map to keep track of it and enable shutdown after all dead
case Supervise(_, _, _) // TODO register child in some map to keep track of it and enable shutdown after all dead
case ChildTerminated(_) stop()
case _ log.error(this + " received unexpected system message [" + message + "]")
}
@ -585,16 +593,17 @@ class LocalActorRefProvider(
if (settings.DebugRouterMisconfiguration && deployer.lookup(path).isDefined)
log.warning("Configuration says that {} should be a router, but code disagrees. Remove the config or add a routerConfig to its Props.")
if (async) new RepointableActorRef(system, props, supervisor, path).initialize()
if (async) new RepointableActorRef(system, props, supervisor, path).initialize(async)
else new LocalActorRef(system, props, supervisor, path)
case router
val lookup = if (lookupDeploy) deployer.lookup(path) else None
val fromProps = Iterator(props.deploy.copy(routerConfig = props.deploy.routerConfig withFallback router))
val d = fromProps ++ deploy.iterator ++ lookup.iterator reduce ((a, b) b withFallback a)
val ref = new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path).initialize()
if (async) ref else ref.activate()
new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path).initialize(async)
}
}
def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None
def getDefaultAddress: Address = rootPath.address
}

View file

@ -613,7 +613,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
protected def createScheduler(): Scheduler =
new DefaultScheduler(
new HashedWheelTimer(log,
threadFactory.copy(threadFactory.name + "-scheduler"),
threadFactory.withName(threadFactory.name + "-scheduler"),
settings.SchedulerTickDuration,
settings.SchedulerTicksPerWheel),
log)

View file

@ -139,16 +139,24 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
}
def parseConfig(key: String, config: Config): Option[Deploy] = {
val deployment = config.withFallback(default)
val router = createRouterConfig(deployment.getString("router"), key, config, deployment)
Some(Deploy(key, deployment, router, NoScopeGiven))
}
/**
* Factory method for creating `RouterConfig`
* @param routerType the configured name of the router, or FQCN
* @param key the full configuration key of the deployment section
* @param config the user defined config of the deployment, without defaults
* @param deployment the deployment config, with defaults
*/
protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = {
val routees = immutableSeq(deployment.getStringList("routees.paths"))
val nrOfInstances = deployment.getInt("nr-of-instances")
val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
val resizer: Option[Resizer] = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
val router: RouterConfig = deployment.getString("router") match {
routerType match {
case "from-code" NoRouter
case "round-robin" RoundRobinRouter(nrOfInstances, routees, resizer)
case "random" RandomRouter(nrOfInstances, routees, resizer)
@ -170,7 +178,6 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
.format(fqn, key), exception)
}).get
}
}
Some(Deploy(key, deployment, router, NoScopeGiven))
}
}

View file

@ -238,7 +238,7 @@ object FSM {
* setTimer("tock", TockMsg, 1 second, true) // repeating
* setTimer("lifetime", TerminateMsg, 1 hour, false) // single-shot
* cancelTimer("tock")
* timerActive_? ("tock")
* isTimerActive("tock")
* </pre>
*/
trait FSM[S, D] extends Listeners with ActorLogging {
@ -372,7 +372,15 @@ trait FSM[S, D] extends Listeners with ActorLogging {
* timer does not exist, has previously been canceled or if it was a
* single-shot timer whose message was already received.
*/
final def timerActive_?(name: String) = timers contains name
@deprecated("use isTimerActive instead", "2.2")
final def timerActive_?(name: String): Boolean = isTimerActive(name)
/**
* Inquire whether the named timer is still active. Returns true unless the
* timer does not exist, has previously been canceled or if it was a
* single-shot timer whose message was already received.
*/
final def isTimerActive(name: String): Boolean = timers contains name
/**
* Set state timeout explicitly. This method can safely be used from within a
@ -380,6 +388,11 @@ trait FSM[S, D] extends Listeners with ActorLogging {
*/
final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout
/**
* Internal API, used for testing.
*/
private[akka] final def isStateTimerActive = timeoutFuture.isDefined
/**
* Set handler which is called upon each state transition, i.e. not when
* staying in the same state. This may use the pair extractor defined in the
@ -634,6 +647,8 @@ trait FSM[S, D] extends Listeners with ActorLogging {
case Failure(msg: AnyRef) log.error(msg.toString)
case _
}
for (timer timers.values) timer.cancel()
timers.clear()
val stopEvent = StopEvent(reason, currentState.stateName, currentState.stateData)
if (terminateEvent.isDefinedAt(stopEvent))
terminateEvent(stopEvent)

View file

@ -4,8 +4,6 @@
package akka.actor
import language.existentials
import akka.dispatch._
import akka.japi.Creator
import scala.reflect.ClassTag

View file

@ -5,17 +5,18 @@
package akka.actor
import java.io.ObjectStreamException
import java.util.{ LinkedList JLinkedList, ListIterator JListIterator }
import java.util.concurrent.TimeUnit
import java.util.concurrent.locks.ReentrantLock
import scala.annotation.tailrec
import scala.collection.mutable.Queue
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.dungeon.ChildrenContainer
import akka.dispatch.{ Envelope, Supervise, SystemMessage, Terminate }
import akka.event.Logging.Warning
import akka.util.Unsafe
import akka.dispatch._
import util.Try
/**
* This actor ref starts out with some dummy cell (by default just enqueuing
@ -32,17 +33,34 @@ private[akka] class RepointableActorRef(
val path: ActorPath)
extends ActorRefWithCell with RepointableRef {
import AbstractActorRef.cellOffset
import AbstractActorRef.{ cellOffset, lookupOffset }
/*
* H E R E B E D R A G O N S !
*
* There are two main functions of a Cell: message queueing and child lookup.
* When switching out the UnstartedCell for its real replacement, the former
* must be switched after all messages have been drained from the temporary
* queue into the real mailbox, while the latter must be switched before
* processing the very first message (i.e. before Cell.start()). Hence there
* are two refs here, one for each function, and they are switched just so.
*/
@volatile private var _cellDoNotCallMeDirectly: Cell = _
@volatile private var _lookupDoNotCallMeDirectly: Cell = _
def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell]
def lookup = Unsafe.instance.getObjectVolatile(this, lookupOffset).asInstanceOf[Cell]
@tailrec final def swapCell(next: Cell): Cell = {
val old = underlying
if (Unsafe.instance.compareAndSwapObject(this, cellOffset, old, next)) old else swapCell(next)
}
@tailrec final def swapLookup(next: Cell): Cell = {
val old = lookup
if (Unsafe.instance.compareAndSwapObject(this, lookupOffset, old, next)) old else swapLookup(next)
}
/**
* Initialize: make a dummy cell which holds just a mailbox, then tell our
* supervisor that we exist so that he can create the real Cell in
@ -52,11 +70,16 @@ private[akka] class RepointableActorRef(
*
* This is protected so that others can have different initialization.
*/
def initialize(): this.type = {
def initialize(async: Boolean): this.type =
underlying match {
case null
val uid = ThreadLocalRandom.current.nextInt()
swapCell(new UnstartedCell(system, this, props, supervisor, uid))
supervisor.sendSystemMessage(Supervise(this, uid))
swapLookup(underlying)
supervisor.sendSystemMessage(Supervise(this, async, uid))
if (!async) point()
this
case other throw new IllegalStateException("initialize called more than once!")
}
/**
@ -65,21 +88,31 @@ private[akka] class RepointableActorRef(
* modification of the `underlying` field, though it is safe to send messages
* at any time.
*/
def activate(): this.type = {
def point(): this.type =
underlying match {
case u: UnstartedCell u.replaceWith(newCell(u))
case _ // this happens routinely for things which were created async=false
}
case u: UnstartedCell
/*
* The problem here was that if the real actor (which will start running
* at cell.start()) creates children in its constructor, then this may
* happen before the swapCell in u.replaceWith, meaning that those
* children cannot be looked up immediately, e.g. if they shall become
* routees.
*/
val cell = newCell(u)
swapLookup(cell)
cell.start()
u.replaceWith(cell)
this
case null throw new IllegalStateException("underlying cell is null")
case _ this // this happens routinely for things which were created async=false
}
/**
* This is called by activate() to obtain the cell which is to replace the
* unstarted cell. The cell must be fully functional.
*/
def newCell(old: Cell): Cell =
new ActorCell(system, this, props, supervisor).
init(old.asInstanceOf[UnstartedCell].uid, sendSupervise = false).start()
def newCell(old: UnstartedCell): Cell =
new ActorCell(system, this, props, supervisor).init(old.uid, sendSupervise = false)
def start(): Unit = ()
@ -91,7 +124,11 @@ private[akka] class RepointableActorRef(
def restart(cause: Throwable): Unit = underlying.restart(cause)
def isStarted: Boolean = !underlying.isInstanceOf[UnstartedCell]
def isStarted: Boolean = underlying match {
case _: UnstartedCell false
case null throw new IllegalStateException("isStarted called before initialized")
case _ true
}
def isTerminated: Boolean = underlying.isTerminated
@ -107,7 +144,7 @@ private[akka] class RepointableActorRef(
case ".." getParent.getChild(name)
case "" getChild(name)
case other
underlying.getChildByName(other) match {
lookup.getChildByName(other) match {
case Some(crs: ChildRestartStats) crs.child.asInstanceOf[InternalActorRef].getChild(name)
case _ Nobody
}
@ -122,117 +159,116 @@ private[akka] class RepointableActorRef(
protected def writeReplace(): AnyRef = SerializedActorRef(path)
}
private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: RepointableActorRef, val props: Props, val supervisor: InternalActorRef, val uid: Int)
extends Cell {
private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl,
val self: RepointableActorRef,
val props: Props,
val supervisor: InternalActorRef,
val uid: Int) extends Cell {
/*
* This lock protects all accesses to this cells queues. It also ensures
* safe switching to the started ActorCell.
*/
val lock = new ReentrantLock
private[this] final val lock = new ReentrantLock
// use Envelope to keep on-send checks in the same place
val queue: Queue[Envelope] = Queue()
val systemQueue: Queue[SystemMessage] = Queue()
var suspendCount: Int = 0
// use Envelope to keep on-send checks in the same place ACCESS MUST BE PROTECTED BY THE LOCK
private[this] final val queue = new JLinkedList[Any]()
private def timeout = system.settings.UnstartedPushTimeout.duration.toMillis
import systemImpl.settings.UnstartedPushTimeout.{ duration timeout }
def replaceWith(cell: Cell): Unit = {
lock.lock()
def replaceWith(cell: Cell): Unit = locked {
try {
/*
* The CallingThreadDispatcher nicely dives under the ReentrantLock and
* breaks things by enqueueing into stale queues from within the message
* processing which happens in-line for sendSystemMessage() and tell().
* Since this is the only possible way to f*ck things up within this
* lock, double-tap (well, N-tap, really); concurrent modification is
* still not possible because were the only thread accessing the queues.
*/
while (systemQueue.nonEmpty || queue.nonEmpty) {
while (systemQueue.nonEmpty) {
val msg = systemQueue.dequeue()
cell.sendSystemMessage(msg)
}
if (queue.nonEmpty) {
val envelope = queue.dequeue()
cell.tell(envelope.message, envelope.sender)
while (!queue.isEmpty) {
queue.poll() match {
case s: SystemMessage cell.sendSystemMessage(s)
case e: Envelope cell.tell(e.message, e.sender)
}
}
} finally try
} finally {
self.swapCell(cell)
finally try
for (_ 1 to suspendCount) cell.suspend()
finally
lock.unlock()
}
}
def system: ActorSystem = systemImpl
def suspend(): Unit = {
lock.lock()
try suspendCount += 1
finally lock.unlock()
}
def resume(causedByFailure: Throwable): Unit = {
lock.lock()
try suspendCount -= 1
finally lock.unlock()
}
def restart(cause: Throwable): Unit = {
lock.lock()
try suspendCount -= 1
finally lock.unlock()
}
def start(): this.type = this
def suspend(): Unit = sendSystemMessage(Suspend())
def resume(causedByFailure: Throwable): Unit = sendSystemMessage(Resume(causedByFailure))
def restart(cause: Throwable): Unit = sendSystemMessage(Recreate(cause))
def stop(): Unit = sendSystemMessage(Terminate())
def isTerminated: Boolean = false
def isTerminated: Boolean = locked {
val cell = self.underlying
if (cellIsReady(cell)) cell.isTerminated else false
}
def parent: InternalActorRef = supervisor
def childrenRefs: ChildrenContainer = ChildrenContainer.EmptyChildrenContainer
def getChildByName(name: String): Option[ChildRestartStats] = None
def tell(message: Any, sender: ActorRef): Unit = {
val useSender = if (sender eq Actor.noSender) system.deadLetters else sender
if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
if (lock.tryLock(timeout.length, timeout.unit)) {
try {
if (self.underlying eq this) queue enqueue Envelope(message, useSender, system)
else self.underlying.tell(message, useSender)
} finally {
lock.unlock()
val cell = self.underlying
if (cellIsReady(cell)) {
cell.tell(message, useSender)
} else if (!queue.offer(Envelope(message, useSender, system))) {
system.eventStream.publish(Warning(self.path.toString, getClass, "dropping message of type " + message.getClass + " due to enqueue failure"))
system.deadLetters ! DeadLetter(message, useSender, self)
}
} finally lock.unlock()
} else {
system.eventStream.publish(Warning(self.path.toString, getClass, "dropping message of type" + message.getClass + " due to lock timeout"))
system.deadLetters ! DeadLetter(message, useSender, self)
}
}
def sendSystemMessage(msg: SystemMessage): Unit = {
if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
try {
if (self.underlying eq this) systemQueue enqueue msg
else self.underlying.sendSystemMessage(msg)
} finally {
lock.unlock()
}
} else {
// FIXME: once we have guaranteed delivery of system messages, hook this in!
system.eventStream.publish(Warning(self.path.toString, getClass, "dropping system message " + msg + " due to lock timeout"))
def sendSystemMessage(msg: SystemMessage): Unit =
if (lock.tryLock(timeout.length, timeout.unit)) {
try {
val cell = self.underlying
if (cellIsReady(cell)) {
cell.sendSystemMessage(msg)
} else {
// systemMessages that are sent during replace need to jump to just after the last system message in the queue, so it's processed before other messages
val wasEnqueued = if ((self.lookup ne this) && (self.underlying eq this) && !queue.isEmpty()) {
@tailrec def tryEnqueue(i: JListIterator[Any] = queue.listIterator(), insertIntoIndex: Int = -1): Boolean =
if (i.hasNext())
tryEnqueue(i,
if (i.next().isInstanceOf[SystemMessage]) i.nextIndex() // update last sysmsg seen so far
else insertIntoIndex) // or just keep the last seen one
else if (insertIntoIndex == -1) queue.offer(msg)
else Try(queue.add(insertIntoIndex, msg)).isSuccess
tryEnqueue()
} else queue.offer(msg)
if (!wasEnqueued) {
system.eventStream.publish(Warning(self.path.toString, getClass, "dropping system message " + msg + " due to enqueue failure"))
system.deadLetters ! DeadLetter(msg, self, self)
}
}
} finally lock.unlock()
} else {
system.eventStream.publish(Warning(self.path.toString, getClass, "dropping system message " + msg + " due to lock timeout"))
system.deadLetters ! DeadLetter(msg, self, self)
}
def isLocal = true
def hasMessages: Boolean = {
private[this] final def cellIsReady(cell: Cell): Boolean = (cell ne this) && (cell ne null)
def hasMessages: Boolean = locked {
val cell = self.underlying
if (cellIsReady(cell)) cell.hasMessages else !queue.isEmpty
}
def numberOfMessages: Int = locked {
val cell = self.underlying
if (cellIsReady(cell)) cell.numberOfMessages else queue.size
}
private[this] final def locked[T](body: T): T = {
lock.lock()
try {
if (self.underlying eq this) !queue.isEmpty
else self.underlying.hasMessages
} finally {
lock.unlock()
}
}
def numberOfMessages: Int = {
lock.lock()
try {
if (self.underlying eq this) queue.size
else self.underlying.numberOfMessages
} finally {
lock.unlock()
}
try body finally lock.unlock()
}
}

View file

@ -15,6 +15,7 @@ import akka.util.internal._
import concurrent.ExecutionContext
import scala.concurrent.duration.FiniteDuration
// The Scheduler trait is included in the documentation. KEEP THE LINES SHORT!!!
//#scheduler
/**
* An Akka scheduler service. This one needs one special behavior: if
@ -38,7 +39,7 @@ trait Scheduler {
initialDelay: FiniteDuration,
interval: FiniteDuration,
receiver: ActorRef,
message: Any)(implicit executor: ExecutionContext): Cancellable
message: Any)(implicit executor: ExecutionContext, sender: ActorRef = Actor.noSender): Cancellable
/**
* Schedules a function to be run repeatedly with an initial delay and a
@ -50,7 +51,8 @@ trait Scheduler {
*/
def schedule(
initialDelay: FiniteDuration,
interval: FiniteDuration)(f: Unit)(implicit executor: ExecutionContext): Cancellable
interval: FiniteDuration)(f: Unit)(
implicit executor: ExecutionContext): Cancellable
/**
* Schedules a function to be run repeatedly with an initial delay and
@ -93,7 +95,8 @@ trait Scheduler {
* Scala API
*/
def scheduleOnce(
delay: FiniteDuration)(f: Unit)(implicit executor: ExecutionContext): Cancellable
delay: FiniteDuration)(f: Unit)(
implicit executor: ExecutionContext): Cancellable
}
//#scheduler
@ -133,7 +136,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter)
override def schedule(initialDelay: FiniteDuration,
delay: FiniteDuration,
receiver: ActorRef,
message: Any)(implicit executor: ExecutionContext): Cancellable = {
message: Any)(implicit executor: ExecutionContext, sender: ActorRef = Actor.noSender): Cancellable = {
val continuousCancellable = new ContinuousCancellable
continuousCancellable.init(
hashedWheelTimer.newTimeout(

View file

@ -128,9 +128,13 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
case class MethodCall(method: Method, parameters: Array[AnyRef]) {
def isOneWay = method.getReturnType == java.lang.Void.TYPE
def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType)
def returnsJOption_? = classOf[akka.japi.Option[_]].isAssignableFrom(method.getReturnType)
def returnsOption_? = classOf[scala.Option[_]].isAssignableFrom(method.getReturnType)
def returnsFuture = classOf[Future[_]] isAssignableFrom method.getReturnType
def returnsJOption = classOf[akka.japi.Option[_]] isAssignableFrom method.getReturnType
def returnsOption = classOf[scala.Option[_]] isAssignableFrom method.getReturnType
@deprecated("use returnsFuture instead", "2.2") def returnsFuture_? = returnsFuture
@deprecated("use returnsJOption instead", "2.2") def returnsJOption_? = returnsJOption
@deprecated("use returnsOption instead", "2.2") def returnsOption_? = returnsOption
/**
* Invokes the Method on the supplied instance
@ -196,6 +200,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
private val selfReference = new ThreadLocal[AnyRef]
private val currentContext = new ThreadLocal[ActorContext]
@SerialVersionUID(1L)
private case object NullResponse
/**
* Returns the reference to the proxy when called inside a method call in a TypedActor
*
@ -296,14 +303,17 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
if (m.isOneWay) m(me)
else {
try {
if (m.returnsFuture_?) {
val s = sender
m(me).asInstanceOf[Future[Any]] onComplete {
m(me) match {
case f: Future[_] if m.returnsFuture
implicit val dispatcher = context.dispatcher
f onComplete {
case Success(null) s ! NullResponse
case Success(result) s ! result
case Failure(f) s ! Status.Failure(f)
case Success(r) s ! r
}
} else {
sender ! m(me)
case null s ! NullResponse
case result s ! result
}
} catch {
case NonFatal(e)
@ -391,6 +401,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
* INTERNAL USE ONLY
*/
private[akka] class TypedActorInvocationHandler(@transient val extension: TypedActorExtension, @transient val actorVar: AtomVar[ActorRef], @transient val timeout: Timeout) extends InvocationHandler with Serializable {
def actor = actorVar.get
@throws(classOf[Throwable])
def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = method.getName match {
@ -398,17 +409,24 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
case "equals" (args.length == 1 && (proxy eq args(0)) || actor == extension.getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean
case "hashCode" actor.hashCode.asInstanceOf[AnyRef]
case _
implicit val dispatcher = extension.system.dispatcher
import akka.pattern.ask
MethodCall(method, args) match {
case m if m.isOneWay actor ! m; null //Null return value
case m if m.returnsFuture_? ask(actor, m)(timeout)
case m if m.returnsJOption_? || m.returnsOption_?
case m if m.returnsFuture ask(actor, m)(timeout) map {
case NullResponse null
case other other
}
case m if m.returnsJOption || m.returnsOption
val f = ask(actor, m)(timeout)
(try { Await.ready(f, timeout.duration).value } catch { case _: TimeoutException None }) match {
case None | Some(Success(null)) if (m.returnsJOption_?) JOption.none[Any] else None
case None | Some(Success(NullResponse)) if (m.returnsJOption) JOption.none[Any] else None
case Some(t: Try[_]) t.get.asInstanceOf[AnyRef]
}
case m Await.result(ask(actor, m)(timeout), timeout.duration).asInstanceOf[AnyRef]
case m Await.result(ask(actor, m)(timeout), timeout.duration) match {
case NullResponse null
case other other.asInstanceOf[AnyRef]
}
}
}
@throws(classOf[ObjectStreamException]) private def writeReplace(): AnyRef = SerializedTypedActorInvocationHandler(actor, timeout.duration)
@ -605,7 +623,7 @@ case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFac
override def isTypedActor(proxyOrNot: AnyRef): Boolean = typedActor.isTypedActor(proxyOrNot)
}
class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory with Extension {
class TypedActorExtension(val system: ExtendedActorSystem) extends TypedActorFactory with Extension {
import TypedActor._ //Import the goodies from the companion object
protected def actorFactory: ActorRefFactory = system
protected def typedActor = this
@ -655,8 +673,8 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory
/**
* INTERNAL USE ONLY
*/
private[akka] def invocationHandlerFor(typedActor_? : AnyRef): TypedActorInvocationHandler =
if ((typedActor_? ne null) && Proxy.isProxyClass(typedActor_?.getClass)) typedActor_? match {
private[akka] def invocationHandlerFor(@deprecatedName('typedActor_?) typedActor: AnyRef): TypedActorInvocationHandler =
if ((typedActor ne null) && Proxy.isProxyClass(typedActor.getClass)) typedActor match {
case null null
case other Proxy.getInvocationHandler(other) match {
case null null

View file

@ -52,19 +52,24 @@ private[akka] trait Children { this: ActorCell ⇒
}
final def stop(actor: ActorRef): Unit = {
val started = actor match {
if (childrenRefs.getByRef(actor).isDefined) {
@tailrec def shallDie(ref: ActorRef): Boolean = {
val c = childrenRefs
swapChildrenRefs(c, c.shallDie(ref)) || shallDie(ref)
}
if (actor match {
case r: RepointableRef r.isStarted
case _ true
}) shallDie(actor)
}
if (childrenRefs.getByRef(actor).isDefined && started) shallDie(actor)
actor.asInstanceOf[InternalActorRef].stop()
}
/*
* low level CAS helpers
*/
@inline private def swapChildrenRefs(oldChildren: ChildrenContainer, newChildren: ChildrenContainer): Boolean =
@inline private final def swapChildrenRefs(oldChildren: ChildrenContainer, newChildren: ChildrenContainer): Boolean =
Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.childrenOffset, oldChildren, newChildren)
@tailrec final def reserveChild(name: String): Boolean = {
@ -89,18 +94,6 @@ private[akka] trait Children { this: ActorCell ⇒
}
}
@tailrec final protected def shallDie(ref: ActorRef): Boolean = {
val c = childrenRefs
swapChildrenRefs(c, c.shallDie(ref)) || shallDie(ref)
}
@tailrec final private def removeChild(ref: ActorRef): ChildrenContainer = {
val c = childrenRefs
val n = c.remove(ref)
if (swapChildrenRefs(c, n)) n
else removeChild(ref)
}
@tailrec final protected def setChildrenTerminationReason(reason: ChildrenContainer.SuspendReason): Boolean = {
childrenRefs match {
case c: ChildrenContainer.TerminatingChildrenContainer
@ -143,10 +136,18 @@ private[akka] trait Children { this: ActorCell ⇒
protected def getAllChildStats: immutable.Iterable[ChildRestartStats] = childrenRefs.stats
protected def removeChildAndGetStateChange(child: ActorRef): Option[SuspendReason] = {
childrenRefs match {
@tailrec def removeChild(ref: ActorRef): ChildrenContainer = {
val c = childrenRefs
val n = c.remove(ref)
if (swapChildrenRefs(c, n)) n else removeChild(ref)
}
childrenRefs match { // The match must be performed BEFORE the removeChild
case TerminatingChildrenContainer(_, _, reason)
val newContainer = removeChild(child)
if (!newContainer.isInstanceOf[TerminatingChildrenContainer]) Some(reason) else None
removeChild(child) match {
case _: TerminatingChildrenContainer None
case _ Some(reason)
}
case _
removeChild(child)
None

View file

@ -55,7 +55,7 @@ private[akka] trait Dispatch { this: ActorCell ⇒
if (sendSupervise) {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
parent.sendSystemMessage(akka.dispatch.Supervise(self, uid))
parent.sendSystemMessage(akka.dispatch.Supervise(self, async = false, uid))
parent ! NullMessage // read ScalaDoc of NullMessage to see why
}
this

View file

@ -108,7 +108,7 @@ private[akka] case class Terminate() extends SystemMessage // sent to self from
/**
* INTERNAL API
*/
private[akka] case class Supervise(child: ActorRef, uid: Int) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start
private[akka] case class Supervise(child: ActorRef, async: Boolean, uid: Int) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start
/**
* INTERNAL API
*/
@ -450,7 +450,6 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
}
class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceConfigurator(config, prerequisites) {
import ThreadPoolConfigBuilder.conf_?
val threadPoolConfig: ThreadPoolConfig = createThreadPoolConfigBuilder(config, prerequisites).config
@ -461,15 +460,15 @@ class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPr
.setCorePoolSizeFromFactor(config getInt "core-pool-size-min", config getDouble "core-pool-size-factor", config getInt "core-pool-size-max")
.setMaxPoolSizeFromFactor(config getInt "max-pool-size-min", config getDouble "max-pool-size-factor", config getInt "max-pool-size-max")
.configure(
conf_?(Some(config getInt "task-queue-size") flatMap {
Some(config getInt "task-queue-size") flatMap {
case size if size > 0
Some(config getString "task-queue-type") map {
case "array" ThreadPoolConfig.arrayBlockingQueue(size, false) //TODO config fairness?
case "" | "linked" ThreadPoolConfig.linkedBlockingQueue(size)
case x throw new IllegalArgumentException("[%s] is not a valid task-queue-type [array|linked]!" format x)
}
} map { qf (q: ThreadPoolConfigBuilder) q.setQueueFactory(qf) }
case _ None
})(queueFactory _.setQueueFactory(queueFactory)))
})
}
def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory =
@ -527,7 +526,7 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer
val tf = threadFactory match {
case m: MonitorableThreadFactory
// add the dispatcher id to the thread names
m.copy(m.name + "-" + id)
m.withName(m.name + "-" + id)
case other other
}
new ForkJoinExecutorServiceFactory(

View file

@ -20,7 +20,7 @@ class PinnedDispatcher(
_id: String,
_mailboxType: MailboxType,
_shutdownTimeout: FiniteDuration,
_threadPoolConfig: ThreadPoolConfig = ThreadPoolConfig())
_threadPoolConfig: ThreadPoolConfig)
extends Dispatcher(_prerequisites,
_id,
Int.MaxValue,

View file

@ -92,17 +92,13 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def
val tf = threadFactory match {
case m: MonitorableThreadFactory
// add the dispatcher id to the thread names
m.copy(m.name + "-" + id)
m.withName(m.name + "-" + id)
case other other
}
new ThreadPoolExecutorServiceFactory(tf)
}
}
object ThreadPoolConfigBuilder {
def conf_?[T](opt: Option[T])(fun: (T) ThreadPoolConfigBuilder ThreadPoolConfigBuilder): Option[(ThreadPoolConfigBuilder) ThreadPoolConfigBuilder] = opt map fun
}
/**
* A DSL to configure and create a MessageDispatcher with a ThreadPoolExecutor
*/
@ -183,9 +179,9 @@ object MonitorableThreadFactory {
case class MonitorableThreadFactory(name: String,
daemonic: Boolean,
contextClassLoader: Option[ClassLoader],
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing)
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing,
protected val counter: AtomicLong = new AtomicLong)
extends ThreadFactory with ForkJoinPool.ForkJoinWorkerThreadFactory {
protected val counter = new AtomicLong
def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = {
val t = wire(new MonitorableThreadFactory.AkkaForkJoinWorkerThread(pool))
@ -196,6 +192,8 @@ case class MonitorableThreadFactory(name: String,
def newThread(runnable: Runnable): Thread = wire(new Thread(runnable, name + "-" + counter.incrementAndGet()))
def withName(newName: String): MonitorableThreadFactory = copy(newName)
protected def wire[T <: Thread](t: T): T = {
t.setUncaughtExceptionHandler(exceptionHandler)
t.setDaemon(daemonic)

View file

@ -73,7 +73,15 @@ trait AskSupport {
*
* See [[scala.concurrent.Future]] for a description of `flow`
*/
def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match {
def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef ? message
}
/*
* Implementation class of the ask pattern enrichment of ActorRef
*/
final class AskableActorRef(val actorRef: ActorRef) extends AnyVal {
def ask(message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match {
case ref: InternalActorRef if ref.isTerminated
actorRef ! message
Future.failed[Any](new AskTimeoutException("Recipient[%s] had already been terminated." format actorRef))
@ -88,71 +96,7 @@ trait AskSupport {
case _ Future.failed[Any](new IllegalArgumentException("Unsupported type of ActorRef for the recipient. Question not sent to [%s]" format actorRef))
}
/**
* Implementation detail of the ask pattern enrichment of ActorRef
*/
private[akka] final class AskableActorRef(val actorRef: ActorRef) {
/**
* Sends a message asynchronously and returns a [[scala.concurrent.Future]]
* holding the eventual reply message; this means that the target actor
* needs to send the result to the `sender` reference provided. The Future
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
* given timeout has expired; this is independent from any timeout applied
* while awaiting a result for this future (i.e. in
* `Await.result(..., timeout)`).
*
* <b>Warning:</b>
* When using future callbacks, inside actors you need to carefully avoid closing over
* the containing actors object, i.e. do not call methods or access mutable state
* on the enclosing actor from within the callback. This would break the actor
* encapsulation and may introduce synchronization bugs and race conditions because
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
* there is not yet a way to detect these illegal accesses at compile time.
*
* <b>Recommended usage:</b>
*
* {{{
* flow {
* val f = worker.ask(request)(timeout)
* EnrichedRequest(request, f())
* } pipeTo nextActor
* }}}
*
* See the [[scala.concurrent.Future]] companion object for a description of `flow`
*/
def ask(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout)
/**
* Sends a message asynchronously and returns a [[scala.concurrent.Future]]
* holding the eventual reply message; this means that the target actor
* needs to send the result to the `sender` reference provided. The Future
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
* given timeout has expired; this is independent from any timeout applied
* while awaiting a result for this future (i.e. in
* `Await.result(..., timeout)`).
*
* <b>Warning:</b>
* When using future callbacks, inside actors you need to carefully avoid closing over
* the containing actors object, i.e. do not call methods or access mutable state
* on the enclosing actor from within the callback. This would break the actor
* encapsulation and may introduce synchronization bugs and race conditions because
* the callback will be scheduled concurrently to the enclosing actor. Unfortunately
* there is not yet a way to detect these illegal accesses at compile time.
*
* <b>Recommended usage:</b>
*
* {{{
* flow {
* val f = worker ? request
* EnrichedRequest(request, f())
* } pipeTo nextActor
* }}}
*
* See the [[scala.concurrent.Future]] companion object for a description of `flow`
*/
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout)
}
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = ask(message)(timeout)
}
/**

View file

@ -237,7 +237,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
}
val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
val selfAddress = routeeProvider.context.system.asInstanceOf[ExtendedActorSystem].provider.rootPath.address
val selfAddress = routeeProvider.context.system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
val vnodes =
if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor
else virtualNodesFactor

View file

@ -7,7 +7,6 @@ import language.implicitConversions
import language.postfixOps
import scala.collection.immutable
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
import scala.concurrent.duration._
import akka.actor._
import akka.ConfigurationException
@ -17,12 +16,14 @@ import akka.japi.Util.immutableSeq
import com.typesafe.config.Config
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
import java.util.concurrent.TimeUnit
import akka.event.Logging.Warning
import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.annotation.tailrec
import akka.event.Logging.Warning
/**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
* send a message to on (or more) of these actors.
* send a message to one (or more) of these actors.
*/
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
extends RepointableActorRef(_system, _props, _supervisor, _path) {
@ -36,11 +37,11 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
_props.routerConfig.verifyConfig()
override def newCell(old: Cell): Cell = new RoutedActorCell(system, this, props, supervisor, old.asInstanceOf[UnstartedCell].uid)
override def newCell(old: UnstartedCell): Cell = new RoutedActorCell(system, this, props, supervisor).init(old.uid, sendSupervise = false)
}
private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef, _uid: Int)
private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef)
extends ActorCell(
_system,
_ref,
@ -71,14 +72,12 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
r
}
init(_uid, sendSupervise = false).start()
/*
* end of construction
*/
def applyRoute(sender: ActorRef, message: Any): immutable.Iterable[Destination] = message match {
case _: AutoReceivedMessage Destination(self, self) :: Nil
case _: AutoReceivedMessage Destination(sender, self) :: Nil
case CurrentRoutees sender ! RouterRoutees(_routees); Nil
case msg if route.isDefinedAt(sender, msg) route(sender, message)
case _ Nil
@ -91,7 +90,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
* `RouterConfig.createRoute` and `Resizer.resize`
*/
private[akka] def addRoutees(newRoutees: immutable.Iterable[ActorRef]): Unit = {
_routees = _routees ++ newRoutees
_routees ++= newRoutees
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
newRoutees foreach watch
}
@ -106,32 +105,40 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
_routees = abandonedRoutees.foldLeft(_routees) { (xs, x) unwatch(x); xs.filterNot(_ == x) }
}
/**
* Send the message to the destinations defined by the `route` function.
*
* If the message is a [[akka.routing.RouterEnvelope]] it will be
* unwrapped before sent to the destinations.
*
* When [[akka.routing.CurrentRoutees]] is sent to the RoutedActorRef it
* replies with [[akka.routing.RouterRoutees]].
*
* Resize is triggered when messages are sent to the routees, and the
* resizer is invoked asynchronously, i.e. not necessarily before the
* message has been sent.
*/
override def tell(message: Any, sender: ActorRef): Unit = {
resize()
val s = if (sender eq null) system.deadLetters else sender
val msg = message match {
case wrapped: RouterEnvelope wrapped.message
case m m
}
applyRoute(s, message) match {
case Destination(_, x) :: Nil if x == self super.tell(message, s)
case refs
refs foreach (p
if (p.recipient == self) super.tell(msg, p.sender)
else p.recipient.!(msg)(p.sender))
applyRoute(s, message) foreach {
case Destination(snd, `self`)
super.tell(msg, snd)
case Destination(snd, recipient)
resize() // only resize when the message target is one of the routees
recipient.tell(msg, snd)
}
}
def resize(): Unit = {
def resize(): Unit =
for (r routerConfig.resizer) {
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true))
super.tell(Router.Resize, self)
}
}
}
/**
* This trait represents a router factory: it produces the actual router actor
@ -199,7 +206,9 @@ trait RouterConfig {
/**
* Routers with dynamically resizable number of routees return the [[akka.routing.Resizer]]
* to use.
* to use. The resizer is invoked once when the router is created, before any messages can
* be sent to it. Resize is also triggered when messages are sent to the routees, and the
* resizer is invoked asynchronously, i.e. not necessarily before the message has been sent.
*/
def resizer: Option[Resizer] = None

View file

@ -32,7 +32,7 @@ object Serialization {
private final def configToMap(path: String): Map[String, String] = {
import scala.collection.JavaConverters._
config.getConfig(path).root.unwrapped.asScala.mapValues(_.toString).toMap
config.getConfig(path).root.unwrapped.asScala.toMap map { case (k, v) (k -> v.toString) }
}
}
}
@ -58,16 +58,16 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* using the optional type hint to the Serializer and the optional ClassLoader ot load it into.
* Returns either the resulting object or an Exception if one was thrown.
*/
def deserialize(bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_]]): Try[AnyRef] =
Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
def deserialize[T](bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_ <: T]]): Try[T] =
Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz).asInstanceOf[T])
/**
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
* You can specify an optional ClassLoader to load the object into.
* Returns either the resulting object or an Exception if one was thrown.
*/
def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] =
Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
def deserialize[T](bytes: Array[Byte], clazz: Class[T]): Try[T] =
Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)).asInstanceOf[T])
/**
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.

View file

@ -5,9 +5,11 @@ package akka.serialization
*/
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
import java.util.concurrent.Callable
import akka.util.ClassLoaderObjectInputStream
import akka.actor.ExtendedActorSystem
import scala.util.DynamicVariable
import akka.serialization.JavaSerializer.CurrentSystem
/**
* A Serializer represents a bimap between an object and an array of bytes representing that object.
@ -93,9 +95,22 @@ object JavaSerializer {
* currentSystem.withValue(system) {
* ...code...
* }
*
* or
*
* currentSystem.withValue(system, callable)
*/
val currentSystem = new DynamicVariable[ExtendedActorSystem](null)
val currentSystem = new CurrentSystem
final class CurrentSystem extends DynamicVariable[ExtendedActorSystem](null) {
/**
* Java API
* @param value - the current value under the call to callable.call()
* @param callable - the operation to be performed
* @tparam S - the return type
* @return the result of callable.call()
*/
def withValue[S](value: ExtendedActorSystem, callable: Callable[S]): S = super.withValue[S](value)(callable.call)
}
}
/**

View file

@ -45,6 +45,11 @@ object ByteString {
*/
def apply(string: String, charset: String): ByteString = CompactByteString(string, charset)
/**
* Creates a new ByteString by copying a byte array.
*/
def fromArray(array: Array[Byte]): ByteString = apply(array)
/**
* Creates a new ByteString by copying length bytes starting at offset from
* an Array.
@ -52,6 +57,16 @@ object ByteString {
def fromArray(array: Array[Byte], offset: Int, length: Int): ByteString =
CompactByteString.fromArray(array, offset, length)
/**
* Creates a new ByteString which will contain the UTF-8 representation of the given String
*/
def fromString(string: String): ByteString = apply(string)
/**
* Creates a new ByteString which will contain the representation of the given String in the given charset
*/
def fromString(string: String, charset: String): ByteString = apply(string, charset)
val empty: ByteString = CompactByteString(Array.empty[Byte])
def newBuilder: ByteStringBuilder = new ByteStringBuilder
@ -282,6 +297,12 @@ sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimiz
override def indexWhere(p: Byte Boolean): Int = iterator.indexWhere(p)
override def indexOf[B >: Byte](elem: B): Int = iterator.indexOf(elem)
/**
* JAVA API
* @return this ByteString copied into a byte array
*/
protected[ByteString] def toArray: Array[Byte] = toArray[Byte] // protected[ByteString] == public to Java but hidden to Scala * fnizz *
override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = iterator.toArray
override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit =
iterator.copyToArray(xs, start, len)

View file

@ -9,6 +9,8 @@ import java.util.regex.Pattern
object Helpers {
val isWindows: Boolean = System.getProperty("os.name", "").toLowerCase.indexOf("win") >= 0
def makePattern(s: String): Pattern = Pattern.compile("^\\Q" + s.replace("?", "\\E.\\Q").replace("*", "\\E.*\\Q") + "\\E$")
def compareIdentityHash(a: AnyRef, b: AnyRef): Int = {

View file

@ -32,7 +32,7 @@ private[akka] object SubclassifiedIndex {
val kids = subkeys flatMap (_ addValue value)
if (!(values contains value)) {
values += value
kids :+ ((key, values))
kids :+ ((key, Set(value)))
} else kids
}

View file

@ -3,7 +3,6 @@ package akka.camel
* Thrown to indicate that the actor referenced by an endpoint URI cannot be
* found in the actor system.
*
* @author Martin Krasser
*/
class ActorNotRegisteredException(uri: String) extends RuntimeException {
override def getMessage: String = "Actor [%s] doesn't exist" format uri

View file

@ -5,7 +5,7 @@
package akka.camel
import java.util.{ Map JMap, Set JSet }
import org.apache.camel.{ CamelContext, Message JCamelMessage }
import org.apache.camel.{ CamelContext, Message JCamelMessage, StreamCache }
import akka.AkkaException
import scala.reflect.ClassTag
import scala.util.Try
@ -14,7 +14,6 @@ import akka.dispatch.Mapper
/**
* An immutable representation of a Camel message.
* @author Martin Krasser
*/
case class CamelMessage(body: Any, headers: Map[String, Any]) {
def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap) //for Java
@ -108,7 +107,21 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
* Java API
*
*/
def getBodyAs[T](clazz: Class[T], camelContext: CamelContext): T = camelContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
def getBodyAs[T](clazz: Class[T], camelContext: CamelContext): T = {
val result = camelContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
// to be able to re-read a StreamCache we must "undo" the side effect by resetting the StreamCache
resetStreamCache()
result
}
/**
* Reset StreamCache body. Nothing is done if the body is not a StreamCache.
* See http://camel.apache.org/stream-caching.html
*/
def resetStreamCache(): Unit = body match {
case stream: StreamCache stream.reset
case _
}
/**
* Returns a new CamelMessage with a new body, while keeping the same headers.
@ -138,7 +151,6 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
/**
* Companion object of CamelMessage class.
*
* @author Martin Krasser
*/
object CamelMessage {
@ -182,7 +194,7 @@ object CamelMessage {
/**
* Positive acknowledgement message (used for application-acknowledged message receipts).
* When `autoAck` is set to false in the [[akka.camel.Consumer]], you can send an `Ack` to the sender of the CamelMessage.
* @author Martin Krasser
*
*/
case object Ack {
/** Java API to get the Ack singleton */

View file

@ -13,7 +13,7 @@ import akka.dispatch.Mapper
/**
* Mixed in by Actor implementations that consume message from Camel endpoints.
*
* @author Martin Krasser
*
*/
trait Consumer extends Actor with CamelSupport {
import Consumer._

View file

@ -13,8 +13,6 @@ import org.apache.camel.processor.SendProcessor
/**
* Support trait for producing messages to Camel endpoints.
*
* @author Martin Krasser
*/
trait ProducerSupport extends Actor with CamelSupport {
private[this] var messages = Map[ActorRef, Any]()
@ -66,7 +64,7 @@ trait ProducerSupport extends Actor with CamelSupport {
for (
child producerChild;
(sender, msg) messages
) child.tell(msg, sender)
) child.tell(transformOutgoingMessage(msg), sender)
Map()
}
}
@ -78,7 +76,7 @@ trait ProducerSupport extends Actor with CamelSupport {
case msg
producerChild match {
case Some(child) child forward msg
case Some(child) child forward transformOutgoingMessage(msg)
case None messages += (sender -> msg)
}
}
@ -110,7 +108,7 @@ trait ProducerSupport extends Actor with CamelSupport {
private class ProducerChild(endpoint: Endpoint, processor: SendProcessor) extends Actor {
def receive = {
case msg @ (_: FailureResult | _: MessageResult) context.parent forward msg
case msg produce(endpoint, processor, transformOutgoingMessage(msg), if (oneway) ExchangePattern.InOnly else ExchangePattern.InOut)
case msg produce(endpoint, processor, msg, if (oneway) ExchangePattern.InOnly else ExchangePattern.InOut)
}
/**
* Initiates a message exchange of given <code>pattern</code> with the endpoint specified by
@ -160,20 +158,20 @@ trait Producer extends ProducerSupport { this: Actor ⇒
/**
* For internal use only.
* @author Martin Krasser
*
*/
private case class MessageResult(message: CamelMessage) extends NoSerializationVerificationNeeded
/**
* For internal use only.
* @author Martin Krasser
*
*/
private case class FailureResult(cause: Throwable, headers: Map[String, Any] = Map.empty) extends NoSerializationVerificationNeeded
/**
* A one-way producer.
*
* @author Martin Krasser
*
*/
trait Oneway extends Producer { this: Actor
override def oneway: Boolean = true

View file

@ -6,15 +6,14 @@ package akka.camel.internal
import akka.actor._
import collection.mutable.WeakHashMap
import akka.camel._
import internal.ActivationProtocol._
import akka.camel.internal.ActivationProtocol._
/**
* For internal use only. An actor that tracks activation and de-activation of endpoints.
*/
private[akka] final class ActivationTracker extends Actor with ActorLogging {
val activations = new WeakHashMap[ActorRef, ActivationStateMachine]
private[camel] class ActivationTracker extends Actor with ActorLogging {
val activations = new WeakHashMap[ActorRef, ActivationStateMachine]
/**
* A state machine that keeps track of the endpoint activation status of an actor.
*/
@ -22,7 +21,6 @@ private[akka] final class ActivationTracker extends Actor with ActorLogging {
type State = PartialFunction[ActivationMessage, Unit]
var receive: State = notActivated()
/**
* Not activated state
* @return a partial function that handles messages in the 'not activated' state
@ -68,8 +66,12 @@ private[akka] final class ActivationTracker extends Actor with ActorLogging {
* @return a partial function that handles messages in the 'de-activated' state
*/
def deactivated: State = {
// deactivated means it was activated at some point, so tell sender it was activated
case AwaitActivation(ref) sender ! EndpointActivated(ref)
case AwaitDeActivation(ref) sender ! EndpointDeActivated(ref)
//resurrected at restart.
case msg @ EndpointActivated(ref)
receive = activated(Nil)
}
/**
@ -80,6 +82,7 @@ private[akka] final class ActivationTracker extends Actor with ActorLogging {
def failedToActivate(cause: Throwable): State = {
case AwaitActivation(ref) sender ! EndpointFailedToActivate(ref, cause)
case AwaitDeActivation(ref) sender ! EndpointFailedToActivate(ref, cause)
case EndpointDeActivated(_) // the de-register at termination always sends a de-activated when the cleanup is done. ignoring.
}
/**
@ -90,6 +93,7 @@ private[akka] final class ActivationTracker extends Actor with ActorLogging {
def failedToDeActivate(cause: Throwable): State = {
case AwaitActivation(ref) sender ! EndpointActivated(ref)
case AwaitDeActivation(ref) sender ! EndpointFailedToDeActivate(ref, cause)
case EndpointDeActivated(_) // the de-register at termination always sends a de-activated when the cleanup is done. ignoring.
}
}

View file

@ -11,7 +11,7 @@ import akka.camel.{ FailureResult, AkkaCamelException, CamelMessage }
* This adapter is used to convert to immutable messages to be used with Actors, and convert the immutable messages back
* to org.apache.camel.Message when using Camel.
*
* @author Martin Krasser
*
*/
private[camel] class CamelExchangeAdapter(val exchange: Exchange) {
/**

View file

@ -115,9 +115,9 @@ private[camel] class Registry(activationTracker: ActorRef) extends Actor with Ca
case msg @ Register(producer, _, None)
if (!producers(producer)) {
producers += producer
producerRegistrar forward msg
parent ! AddWatch(producer)
}
producerRegistrar forward msg
case DeRegister(actorRef)
producers.find(_ == actorRef).foreach { p
deRegisterProducer(p)
@ -155,6 +155,8 @@ private[camel] class ProducerRegistrar(activationTracker: ActorRef) extends Acto
} catch {
case NonFatal(e) throw new ActorActivationException(producer, e)
}
} else {
camelObjects.get(producer).foreach { case (endpoint, processor) producer ! CamelProducerObjects(endpoint, processor) }
}
case DeRegister(producer)
camelObjects.get(producer).foreach {

View file

@ -16,7 +16,7 @@ import org.apache.camel.model.RouteDefinition
*
* @param endpointUri endpoint URI of the consumer actor.
*
* @author Martin Krasser
*
*/
private[camel] class ConsumerActorRouteBuilder(endpointUri: String, consumer: ActorRef, config: ConsumerConfig, settings: CamelSettings) extends RouteBuilder {

View file

@ -31,7 +31,7 @@ import scala.util.{ Failure, Success, Try }
* Messages are sent to [[akka.camel.Consumer]] actors through a [[akka.camel.internal.component.ActorEndpoint]] that
* this component provides.
*
* @author Martin Krasser
*
*/
private[camel] class ActorComponent(camel: Camel, system: ActorSystem) extends DefaultComponent {
/**
@ -52,7 +52,7 @@ private[camel] class ActorComponent(camel: Camel, system: ActorSystem) extends D
* <code>[actorPath]?[options]%s</code>,
* where <code>[actorPath]</code> refers to the actor path to the actor.
*
* @author Martin Krasser
*
*/
private[camel] class ActorEndpoint(uri: String,
comp: ActorComponent,
@ -104,7 +104,7 @@ private[camel] trait ActorEndpointConfig {
* @see akka.camel.component.ActorComponent
* @see akka.camel.component.ActorEndpoint
*
* @author Martin Krasser
*
*/
private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) extends DefaultProducer(endpoint) with AsyncProcessor {
/**

View file

@ -12,7 +12,7 @@ import org.apache.camel.impl.DefaultCamelContext
/**
* Subclass this abstract class to create an untyped producer actor. This class is meant to be used from Java.
*
* @author Martin Krasser
*
*/
abstract class UntypedProducerActor extends UntypedActor with ProducerSupport {
/**

View file

@ -19,7 +19,7 @@ import java.util.concurrent.TimeUnit;
import akka.testkit.AkkaSpec;
import static org.junit.Assert.*;
/**
* @author Martin Krasser
*
*/
public class ConsumerJavaTestBase {

View file

@ -8,6 +8,7 @@ import akka.actor.ActorSystem;
import akka.dispatch.Mapper;
import akka.japi.Function;
import org.apache.camel.NoTypeConversionAvailableException;
import org.apache.camel.converter.stream.InputStreamCache;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -18,7 +19,7 @@ import java.util.*;
import static org.junit.Assert.assertEquals;
/**
* @author Martin Krasser
*
*/
public class MessageJavaTestBase {
static Camel camel;
@ -100,6 +101,14 @@ public class MessageJavaTestBase {
message("test1" , createMap("A", "1")).withHeaders(createMap("C", "3")));
}
@Test
public void shouldBeAbleToReReadStreamCacheBody() throws Exception {
CamelMessage msg = new CamelMessage(new InputStreamCache("test1".getBytes("utf-8")), empty);
assertEquals("test1", msg.getBodyAs(String.class, camel.context()));
// re-read
assertEquals("test1", msg.getBodyAs(String.class, camel.context()));
}
private static Set<String> createSet(String... entries) {
HashSet<String> set = new HashSet<String>();
set.addAll(Arrays.asList(entries));

View file

@ -15,7 +15,7 @@ import scala.Option;
import scala.concurrent.duration.FiniteDuration;
/**
* @author Martin Krasser
*
*/
public class SampleErrorHandlingConsumer extends UntypedConsumerActor {
private static Mapper<RouteDefinition, ProcessorDefinition<?>> mapper = new Mapper<RouteDefinition, ProcessorDefinition<?>>() {

View file

@ -7,7 +7,7 @@ package akka.camel;
import akka.camel.javaapi.UntypedConsumerActor;
/**
* @author Martin Krasser
*
*/
public class SampleUntypedConsumer extends UntypedConsumerActor {

View file

@ -6,7 +6,7 @@ package akka.camel;
import akka.camel.javaapi.UntypedProducerActor;
/**
* @author Martin Krasser
*
*/
public class SampleUntypedForwardingProducer extends UntypedProducerActor {

View file

@ -7,7 +7,7 @@ package akka.camel;
import akka.camel.javaapi.UntypedProducerActor;
/**
* @author Martin Krasser
*
*/
public class SampleUntypedReplyingProducer extends UntypedProducerActor {

View file

@ -30,7 +30,7 @@ class ConsumerIntegrationTest extends WordSpec with MustMatchers with NonSharedC
"Consumer must throw FailedToCreateRouteException, while awaiting activation, if endpoint is invalid" in {
filterEvents(EventFilter[ActorActivationException](occurrences = 1)) {
val actorRef = system.actorOf(Props(new TestActor(uri = "some invalid uri")))
val actorRef = system.actorOf(Props(new TestActor(uri = "some invalid uri")), "invalidActor")
intercept[FailedToCreateRouteException] {
Await.result(camel.activationFutureFor(actorRef), defaultTimeoutDuration)
}

View file

@ -5,11 +5,11 @@
package akka.camel
import java.io.InputStream
import org.apache.camel.NoTypeConversionAvailableException
import akka.camel.TestSupport.{ SharedCamelSystem }
import org.scalatest.FunSuite
import org.scalatest.matchers.MustMatchers
import org.apache.camel.converter.stream.InputStreamCache
class MessageScalaTest extends FunSuite with MustMatchers with SharedCamelSystem {
implicit def camelContext = camel.context
@ -44,12 +44,17 @@ class MessageScalaTest extends FunSuite with MustMatchers with SharedCamelSystem
test("mustSetBodyAndPreserveHeaders") {
CamelMessage("test1", Map("A" -> "1")).copy(body = "test2") must be(
CamelMessage("test2", Map("A" -> "1")))
}
test("mustSetHeadersAndPreserveBody") {
CamelMessage("test1", Map("A" -> "1")).copy(headers = Map("C" -> "3")) must be(
CamelMessage("test1", Map("C" -> "3")))
}
test("mustBeAbleToReReadStreamCacheBody") {
val msg = CamelMessage(new InputStreamCache("test1".getBytes("utf-8")), Map.empty)
msg.bodyAs[String] must be("test1")
// re-read
msg.bodyAs[String] must be("test1")
}
}

View file

@ -19,13 +19,20 @@ import scala.concurrent.duration._
import akka.util.Timeout
import org.scalatest.matchers.MustMatchers
import akka.testkit._
import akka.actor.Status.Failure
/**
* Tests the features of the Camel Producer.
*/
class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAndAfterEach with SharedCamelSystem with MustMatchers {
class ProducerFeatureTest extends TestKit(ActorSystem("test", AkkaSpec.testConf)) with WordSpec with BeforeAndAfterAll with BeforeAndAfterEach with MustMatchers {
import ProducerFeatureTest._
implicit def camel = CamelExtension(system)
override protected def afterAll() {
super.afterAll()
system.shutdown()
}
val camelContext = camel.context
// to make testing equality of messages easier, otherwise the breadcrumb shows up in the result.
@ -40,9 +47,8 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
"produce a message and receive normal response" in {
val producer = system.actorOf(Props(new TestProducer("direct:producer-test-2", true)), name = "direct-producer-2")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123"))
val future = producer.ask(message)(timeoutDuration)
val expected = CamelMessage("received TEST", Map(CamelMessage.MessageExchangeId -> "123"))
Await.result(future, timeoutDuration) must be === expected
producer.tell(message, testActor)
expectMsg(CamelMessage("received TEST", Map(CamelMessage.MessageExchangeId -> "123")))
stopGracefully(producer)
}
@ -65,13 +71,18 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
case _: AkkaCamelException Stop
}
}), name = "prod-anonymous-supervisor")
val producer = Await.result[ActorRef](supervisor.ask(Props(new TestProducer("direct:producer-test-2"))).mapTo[ActorRef], timeoutDuration)
supervisor.tell(Props(new TestProducer("direct:producer-test-2")), testActor)
val producer = receiveOne(timeoutDuration).asInstanceOf[ActorRef]
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
val e = intercept[AkkaCamelException] { Await.result(producer.ask(message)(timeoutDuration), timeoutDuration) }
producer.tell(message, testActor)
expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException)
e.getMessage must be("failure")
e.headers must be(Map(CamelMessage.MessageExchangeId -> "123"))
}
}
Await.ready(latch, timeoutDuration)
deadActor must be(Some(producer))
stopGracefully(producer)
@ -101,15 +112,8 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
"produce message to direct:producer-test-3 and receive normal response" in {
val producer = system.actorOf(Props(new TestProducer("direct:producer-test-3")), name = "direct-producer-test-3")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123"))
val future = producer.ask(message)(timeoutDuration)
Await.result(future, timeoutDuration) match {
case result: CamelMessage
// a normal response must have been returned by the producer
val expected = CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123"))
result must be(expected)
case unexpected fail("Actor responded with unexpected message:" + unexpected)
}
producer.tell(message, testActor)
expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123")))
stopGracefully(producer)
}
@ -118,10 +122,13 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
val e = intercept[AkkaCamelException] { Await.result(producer.ask(message)(timeoutDuration), timeoutDuration) }
producer.tell(message, testActor)
expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException)
e.getMessage must be("failure")
e.headers must be(Map(CamelMessage.MessageExchangeId -> "123"))
}
}
stopGracefully(producer)
}
@ -129,15 +136,8 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
val target = system.actorOf(Props[ReplyingForwardTarget], name = "reply-forwarding-target")
val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-2", target)), name = "direct-producer-test-2-forwarder")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123"))
val future = producer.ask(message)(timeoutDuration)
Await.result(future, timeoutDuration) match {
case result: CamelMessage
// a normal response must have been returned by the forward target
val expected = CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123", "test" -> "result"))
result must be(expected)
case unexpected fail("Actor responded with unexpected message:" + unexpected)
}
producer.tell(message, testActor)
expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123", "test" -> "result")))
stopGracefully(target, producer)
}
@ -147,10 +147,13 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
val e = intercept[AkkaCamelException] { Await.result(producer.ask(message)(timeoutDuration), timeoutDuration) }
producer.tell(message, testActor)
expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException)
e.getMessage must be("failure")
e.headers must be(Map(CamelMessage.MessageExchangeId -> "123", "test" -> "failure"))
}
}
stopGracefully(target, producer)
}
@ -180,13 +183,8 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-3", target)), name = "direct-producer-test-3-to-replying-actor")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123"))
val future = producer.ask(message)(timeoutDuration)
Await.result(future, timeoutDuration) match {
case message: CamelMessage
val expected = CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123", "test" -> "result"))
message must be(expected)
case unexpected fail("Actor responded with unexpected message:" + unexpected)
}
producer.tell(message, testActor)
expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123", "test" -> "result")))
stopGracefully(target, producer)
}
@ -196,10 +194,13 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
val e = intercept[AkkaCamelException] { Await.result(producer.ask(message)(timeoutDuration), timeoutDuration) }
producer.tell(message, testActor)
expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException)
e.getMessage must be("failure")
e.headers must be(Map(CamelMessage.MessageExchangeId -> "123", "test" -> "failure"))
}
}
stopGracefully(target, producer)
}
@ -224,6 +225,36 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
}
stopGracefully(target, producer)
}
"keep producing messages after error" in {
import TestSupport._
val consumer = start(new IntermittentErrorConsumer("direct:intermittentTest-1"), "intermittentTest-error-consumer")
val producer = start(new SimpleProducer("direct:intermittentTest-1"), "intermittentTest-producer")
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
val futureFailed = producer.tell("fail", testActor)
expectMsgPF(timeoutDuration) {
case Failure(e)
e.getMessage must be("fail")
}
producer.tell("OK", testActor)
expectMsg("OK")
}
stop(consumer)
stop(producer)
}
"be able to transform outgoing messages and have a valid sender reference" in {
import TestSupport._
filterEvents(EventFilter[Exception](occurrences = 1)) {
val producerSupervisor = system.actorOf(Props(new ProducerSupervisor(Props(new ChildProducer("mock:mock", true)))), "ignore-deadletter-sender-ref-test")
mockEndpoint.reset()
producerSupervisor.tell(CamelMessage("test", Map()), testActor)
producerSupervisor.tell(CamelMessage("err", Map()), testActor)
mockEndpoint.expectedMessageCount(1)
mockEndpoint.expectedBodiesReceived("TEST")
expectMsg("TEST")
system.stop(producerSupervisor)
}
}
}
private def mockEndpoint = camel.context.getEndpoint("mock:mock", classOf[MockEndpoint])
@ -236,9 +267,52 @@ class ProducerFeatureTest extends WordSpec with BeforeAndAfterAll with BeforeAnd
}
object ProducerFeatureTest {
class ProducerSupervisor(childProps: Props) extends Actor {
override def supervisorStrategy = SupervisorStrategy.stoppingStrategy
val child = context.actorOf(childProps, "producer-supervisor-child")
val duration = 10 seconds
implicit val timeout = Timeout(duration)
implicit val ec = context.system.dispatcher
Await.ready(CamelExtension(context.system).activationFutureFor(child), timeout.duration)
def receive = {
case msg: CamelMessage
child forward (msg)
case (aref: ActorRef, msg: String)
aref ! msg
}
}
class ChildProducer(uri: String, upper: Boolean = false) extends Actor with Producer {
override def oneway = true
var lastSender: Option[ActorRef] = None
var lastMessage: Option[String] = None
def endpointUri = uri
override def transformOutgoingMessage(msg: Any) = msg match {
case msg: CamelMessage if (upper) msg.mapBody {
body: String
if (body == "err") throw new Exception("Crash!")
val upperMsg = body.toUpperCase
lastSender = Some(sender)
lastMessage = Some(upperMsg)
}
else msg
}
override def postStop() {
for (msg lastMessage; aref lastSender) context.parent ! (aref, msg)
super.postStop()
}
}
class TestProducer(uri: String, upper: Boolean = false) extends Actor with Producer {
def endpointUri = uri
override def preRestart(reason: Throwable, message: Option[Any]) {
//overriding on purpose so it doesn't try to deRegister and reRegister at restart,
// which would cause a deadletter message in the test output.
}
override protected def transformOutgoingMessage(msg: Any) = msg match {
case msg: CamelMessage if (upper) msg.mapBody {
body: String body.toUpperCase
@ -303,4 +377,18 @@ object ProducerFeatureTest {
}
}
class SimpleProducer(override val endpointUri: String) extends Producer {
override protected def transformResponse(msg: Any) = msg match {
case m: CamelMessage m.bodyAs[String]
case m: Any m
}
}
class IntermittentErrorConsumer(override val endpointUri: String) extends Consumer {
def receive = {
case msg: CamelMessage if msg.bodyAs[String] == "fail" sender ! Failure(new Exception("fail"))
case msg: CamelMessage sender ! msg
}
}
}

View file

@ -109,6 +109,14 @@ class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpec w
awaiting.verifyActivated()
}
"send activation message when an actor is activated, deactivated and activated again" taggedAs TimingTest in {
publish(EndpointActivated(actor.ref))
publish(EndpointDeActivated(actor.ref))
publish(EndpointActivated(actor.ref))
awaiting.awaitActivation()
awaiting.verifyActivated()
}
}
class Awaiting(actor: TestProbe) {

View file

@ -28,6 +28,12 @@ akka {
# formed in case of network partition.
auto-down = off
# Minimum required number of members before the leader changes member status
# of 'Joining' members to 'Up'. Typically used together with
# 'Cluster.registerOnMemberUp' to defer some action, such as starting actors,
# until the cluster has reached a certain size.
min-nr-of-members = 1
# Enable or disable JMX MBeans for management of the cluster
jmx.enabled = on
@ -70,7 +76,7 @@ akka {
failure-detector {
# FQCN of the failure detector implementation.
# It must implement akka.cluster.akka.cluster and
# It must implement akka.cluster.FailureDetector and
# have constructor with akka.actor.ActorSystem and
# akka.cluster.ClusterSettings parameters
implementation-class = "akka.cluster.AccrualFailureDetector"
@ -106,22 +112,32 @@ akka {
max-sample-size = 1000
}
# Uses JMX and Hyperic SIGAR, if SIGAR is on the classpath.
metrics {
# Enable or disable metrics collector for load-balancing nodes.
enabled = on
# How often metrics is sampled on a node.
metrics-interval = 3s
# FQCN of the metrics collector implementation.
# It must implement akka.cluster.cluster.MetricsCollector and
# have constructor with akka.actor.ActorSystem parameter.
# The default SigarMetricsCollector uses JMX and Hyperic SIGAR, if SIGAR
# is on the classpath, otherwise only JMX.
collector-class = "akka.cluster.SigarMetricsCollector"
# How often metrics are sampled on a node.
# Shorter interval will collect the metrics more often.
collect-interval = 3s
# How often a node publishes metrics information.
gossip-interval = 3s
# How quickly the exponential weighting of past data is decayed compared to
# new data.
# If set to 0 data streaming over time will be turned off.
# Set higher to increase the bias toward newer values
rate-of-decay = 10
# new data. Set lower to increase the bias toward newer values.
# The relevance of each data sample is halved for every passing half-life duration,
# i.e. after 4 times the half-life, a data samples relevance is reduced to 6% of
# its original relevance. The initial relevance of a data sample is given by
# 1 0.5 ^ (collect-interval / half-life).
# See http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average
moving-average-half-life = 12s
}
# If the tick-duration of the default scheduler is longer than the
@ -143,6 +159,16 @@ akka {
}
# Default configuration for routers
actor.deployment.default {
# MetricsSelector to use
# - available: "mix", "heap", "cpu", "load"
# - or: Fully qualified class name of the MetricsSelector class.
# The class must extend akka.cluster.routing.MetricsSelector
# and have a constructor with com.typesafe.config.Config
# parameter.
# - default is "mix"
metrics-selector = mix
}
actor.deployment.default.cluster {
# enable cluster aware router that deploys to nodes in the cluster
enabled = off
@ -169,4 +195,5 @@ akka {
routees-path = ""
}
}

View file

@ -6,7 +6,7 @@ package akka.cluster
import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
import akka.event.Logging
import scala.collection.immutable.Map
import scala.collection.immutable
import scala.annotation.tailrec
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.TimeUnit.NANOSECONDS
@ -233,7 +233,7 @@ private[cluster] object HeartbeatHistory {
*/
def apply(maxSampleSize: Int): HeartbeatHistory = HeartbeatHistory(
maxSampleSize = maxSampleSize,
intervals = IndexedSeq.empty,
intervals = immutable.IndexedSeq.empty,
intervalSum = 0L,
squaredIntervalSum = 0L)
@ -248,7 +248,7 @@ private[cluster] object HeartbeatHistory {
*/
private[cluster] case class HeartbeatHistory private (
maxSampleSize: Int,
intervals: IndexedSeq[Long],
intervals: immutable.IndexedSeq[Long],
intervalSum: Long,
squaredIntervalSum: Long) {

View file

@ -17,7 +17,7 @@ import akka.util._
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.annotation.tailrec
import scala.collection.immutable.SortedSet
import scala.collection.immutable
import java.io.Closeable
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicReference
@ -67,7 +67,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
format(system, other.getClass.getName))
}
private val _isRunning = new AtomicBoolean(true)
private val _isTerminated = new AtomicBoolean(false)
private val log = Logging(system, "Cluster")
log.info("Cluster Node [{}] - is starting up...", selfAddress)
@ -95,7 +95,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
new DefaultScheduler(
new HashedWheelTimer(log,
system.threadFactory match {
case tf: MonitorableThreadFactory tf.copy(name = tf.name + "-cluster-scheduler")
case tf: MonitorableThreadFactory tf.withName(tf.name + "-cluster-scheduler")
case tf tf
},
SchedulerTickDuration,
@ -108,7 +108,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
override def close(): Unit = () // we are using system.scheduler, which we are not responsible for closing
override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration,
receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable =
receiver: ActorRef, message: Any)(implicit executor: ExecutionContext, sender: ActorRef = Actor.noSender): Cancellable =
systemScheduler.schedule(initialDelay, interval, receiver, message)
override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration)(f: Unit)(implicit executor: ExecutionContext): Cancellable =
@ -169,9 +169,9 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
// ======================================================
/**
* Returns true if the cluster node is up and running, false if it is shut down.
* Returns true if this cluster instance has be shutdown.
*/
def isRunning: Boolean = _isRunning.get
def isTerminated: Boolean = _isTerminated.get
/**
* Subscribe to cluster domain events.
@ -232,6 +232,24 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
def down(address: Address): Unit =
clusterCore ! ClusterUserAction.Down(address)
/**
* The supplied thunk will be run, once, when current cluster member is `Up`.
* Typically used together with configuration option `akka.cluster.min-nr-of-members'
* to defer some action, such as starting actors, until the cluster has reached
* a certain size.
*/
def registerOnMemberUp[T](code: T): Unit =
registerOnMemberUp(new Runnable { def run = code })
/**
* The supplied callback will be run, once, when current cluster member is `Up`.
* Typically used together with configuration option `akka.cluster.min-nr-of-members'
* to defer some action, such as starting actors, until the cluster has reached
* a certain size.
* JAVA API
*/
def registerOnMemberUp(callback: Runnable): Unit = clusterDaemons ! InternalClusterAction.AddOnMemberUpListener(callback)
// ========================================================
// ===================== INTERNAL API =====================
// ========================================================
@ -241,7 +259,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
* in config. Especially useful from tests when Addresses are unknown
* before startup time.
*/
private[cluster] def joinSeedNodes(seedNodes: IndexedSeq[Address]): Unit =
private[cluster] def joinSeedNodes(seedNodes: immutable.IndexedSeq[Address]): Unit =
clusterCore ! InternalClusterAction.JoinSeedNodes(seedNodes)
/**
@ -253,7 +271,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
* to go through graceful handoff process `LEAVE -> EXITING -> REMOVED -> SHUTDOWN`.
*/
private[cluster] def shutdown(): Unit = {
if (_isRunning.compareAndSet(true, false)) {
if (_isTerminated.compareAndSet(false, true)) {
log.info("Cluster Node [{}] - Shutting down cluster Node and cluster daemons...", selfAddress)
system.stop(clusterDaemons)
@ -268,4 +286,3 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
}
}

View file

@ -18,16 +18,29 @@ import akka.actor.Props
import akka.actor.Scheduler
import akka.actor.Scope
import akka.actor.Terminated
import akka.cluster.routing.ClusterRouterConfig
import akka.cluster.routing.ClusterRouterSettings
import akka.dispatch.ChildTerminated
import akka.event.EventStream
import akka.japi.Util.immutableSeq
import akka.remote.RemoteActorRefProvider
import akka.remote.RemoteDeployer
import akka.remote.routing.RemoteRouterConfig
import akka.routing.RouterConfig
import akka.routing.DefaultResizer
import akka.cluster.routing.ClusterRouterConfig
import akka.cluster.routing.ClusterRouterSettings
import akka.cluster.routing.AdaptiveLoadBalancingRouter
import akka.cluster.routing.MixMetricsSelector
import akka.cluster.routing.HeapMetricsSelector
import akka.cluster.routing.SystemLoadAverageMetricsSelector
import akka.cluster.routing.CpuMetricsSelector
import akka.cluster.routing.MetricsSelector
/**
* INTERNAL API
*
* The `ClusterActorRefProvider` will load the [[akka.cluster.Cluster]]
* extension, i.e. the cluster will automatically be started when
* the `ClusterActorRefProvider` is used.
*/
class ClusterActorRefProvider(
_systemName: String,
@ -42,10 +55,17 @@ class ClusterActorRefProvider(
override def init(system: ActorSystemImpl): Unit = {
super.init(system)
// initialize/load the Cluster extension
Cluster(system)
remoteDeploymentWatcher = system.systemActorOf(Props[RemoteDeploymentWatcher], "RemoteDeploymentWatcher")
}
override val deployer: ClusterDeployer = new ClusterDeployer(settings, dynamicAccess)
/**
* Factory method to make it possible to override deployer in subclass
* Creates a new instance every time
*/
override protected def createDeployer: ClusterDeployer = new ClusterDeployer(settings, dynamicAccess)
/**
* This method is overridden here to keep track of remote deployed actors to
@ -108,6 +128,36 @@ private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: Dynami
case None None
}
}
override protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = {
val routees = immutableSeq(deployment.getStringList("routees.paths"))
val nrOfInstances = deployment.getInt("nr-of-instances")
val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
routerType match {
case "adaptive"
val metricsSelector = deployment.getString("metrics-selector") match {
case "mix" MixMetricsSelector
case "heap" HeapMetricsSelector
case "cpu" CpuMetricsSelector
case "load" SystemLoadAverageMetricsSelector
case fqn
val args = List(classOf[Config] -> deployment)
dynamicAccess.createInstanceFor[MetricsSelector](fqn, args).recover({
case exception throw new IllegalArgumentException(
("Cannot instantiate metrics-selector [%s], defined in [%s], " +
"make sure it extends [akka.cluster.routing.MetricsSelector] and " +
"has constructor with [com.typesafe.config.Config] parameter")
.format(fqn, key), exception)
}).get
}
AdaptiveLoadBalancingRouter(metricsSelector, nrOfInstances, routees, resizer)
case _ super.createRouterConfig(routerType, key, config, deployment)
}
}
}
@SerialVersionUID(1L)

View file

@ -3,9 +3,12 @@
*/
package akka.cluster
import scala.collection.immutable.SortedSet
import language.existentials
import language.postfixOps
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.util.control.NonFatal
import akka.actor.{ Actor, ActorLogging, ActorRef, Address, Cancellable, Props, ReceiveTimeout, RootActorPath, Scheduler }
import akka.actor.Status.Failure
import akka.event.EventStream
@ -13,8 +16,6 @@ import akka.pattern.ask
import akka.util.Timeout
import akka.cluster.MemberStatus._
import akka.cluster.ClusterEvent._
import language.existentials
import language.postfixOps
/**
* Base trait for all cluster messages. All ClusterMessage's are serializable.
@ -61,7 +62,7 @@ private[cluster] object InternalClusterAction {
* Command to initiate the process to join the specified
* seed nodes.
*/
case class JoinSeedNodes(seedNodes: IndexedSeq[Address])
case class JoinSeedNodes(seedNodes: immutable.IndexedSeq[Address])
/**
* Start message of the process to join one of the seed nodes.
@ -104,6 +105,12 @@ private[cluster] object InternalClusterAction {
case object GetClusterCoreRef
/**
* Comand to [[akka.cluster.ClusterDaemon]] to create a
* [[akka.cluster.OnMemberUpListener]].
*/
case class AddOnMemberUpListener(callback: Runnable)
sealed trait SubscriptionMessage
case class Subscribe(subscriber: ActorRef, to: Class[_]) extends SubscriptionMessage
case class Unsubscribe(subscriber: ActorRef, to: Option[Class[_]]) extends SubscriptionMessage
@ -113,10 +120,12 @@ private[cluster] object InternalClusterAction {
*/
case class PublishCurrentClusterState(receiver: Option[ActorRef]) extends SubscriptionMessage
case class PublishChanges(oldGossip: Gossip, newGossip: Gossip)
case class PublishEvent(event: ClusterDomainEvent)
case object PublishDone
sealed trait PublishMessage
case class PublishChanges(newGossip: Gossip) extends PublishMessage
case class PublishEvent(event: ClusterDomainEvent) extends PublishMessage
case object PublishStart extends PublishMessage
case object PublishDone extends PublishMessage
case object PublishDoneFinished extends PublishMessage
}
/**
@ -160,6 +169,8 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac
def receive = {
case InternalClusterAction.GetClusterCoreRef sender ! core
case InternalClusterAction.AddOnMemberUpListener(code)
context.actorOf(Props(new OnMemberUpListener(code)))
}
}
@ -256,7 +267,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
def initJoin(): Unit = sender ! InitJoinAck(selfAddress)
def joinSeedNodes(seedNodes: IndexedSeq[Address]): Unit = {
def joinSeedNodes(seedNodes: immutable.IndexedSeq[Address]): Unit = {
// only the node which is named first in the list of seed nodes will join itself
if (seedNodes.isEmpty || seedNodes.head == selfAddress)
self ! JoinTo(selfAddress)
@ -271,14 +282,14 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
*/
def join(address: Address): Unit = {
if (!latestGossip.members.exists(_.address == address)) {
val localGossip = latestGossip
// wipe our state since a node that joins a cluster must be empty
latestGossip = Gossip()
// wipe the failure detector since we are starting fresh and shouldn't care about the past
failureDetector.reset()
// wipe the publisher since we are starting fresh
publisher ! PublishStart
publish(localGossip)
publish(latestGossip)
heartbeatSender ! JoinInProgress(address, Deadline.now + JoinTimeout)
context.become(initialized)
@ -293,18 +304,16 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
* State transition to JOINING - new node joining.
*/
def joining(node: Address): Unit = {
val localGossip = latestGossip
val localMembers = localGossip.members
val localUnreachable = localGossip.overview.unreachable
val localMembers = latestGossip.members
val localUnreachable = latestGossip.overview.unreachable
val alreadyMember = localMembers.exists(_.address == node)
val isUnreachable = localGossip.overview.isNonDownUnreachable(node)
val isUnreachable = latestGossip.overview.isNonDownUnreachable(node)
if (!alreadyMember && !isUnreachable) {
// remove the node from the 'unreachable' set in case it is a DOWN node that is rejoining cluster
val (rejoiningMember, newUnreachableMembers) = localUnreachable partition { _.address == node }
val newOverview = localGossip.overview copy (unreachable = newUnreachableMembers)
val newOverview = latestGossip.overview copy (unreachable = newUnreachableMembers)
// remove the node from the failure detector if it is a DOWN node that is rejoining cluster
if (rejoiningMember.nonEmpty) failureDetector.remove(node)
@ -312,7 +321,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
// add joining node as Joining
// add self in case someone else joins before self has joined (Set discards duplicates)
val newMembers = localMembers + Member(node, Joining) + Member(selfAddress, Joining)
val newGossip = localGossip copy (overview = newOverview, members = newMembers)
val newGossip = latestGossip copy (overview = newOverview, members = newMembers)
val versionedGossip = newGossip :+ vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
@ -326,7 +335,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
gossipTo(node)
}
publish(localGossip)
publish(latestGossip)
}
}
@ -334,10 +343,9 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
* State transition to LEAVING.
*/
def leaving(address: Address): Unit = {
val localGossip = latestGossip
if (localGossip.members.exists(_.address == address)) { // only try to update if the node is available (in the member ring)
val newMembers = localGossip.members map { member if (member.address == address) Member(address, Leaving) else member } // mark node as LEAVING
val newGossip = localGossip copy (members = newMembers)
if (latestGossip.members.exists(_.address == address)) { // only try to update if the node is available (in the member ring)
val newMembers = latestGossip.members map { member if (member.address == address) Member(address, Leaving) else member } // mark node as LEAVING
val newGossip = latestGossip copy (members = newMembers)
val versionedGossip = newGossip :+ vclockNode
val seenVersionedGossip = versionedGossip seen selfAddress
@ -345,7 +353,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
latestGossip = seenVersionedGossip
log.info("Cluster Node [{}] - Marked address [{}] as LEAVING", selfAddress, address)
publish(localGossip)
publish(latestGossip)
}
}
@ -368,10 +376,9 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
*/
def removing(address: Address): Unit = {
log.info("Cluster Node [{}] - Node has been REMOVED by the leader - shutting down...", selfAddress)
val localGossip = latestGossip
// just cleaning up the gossip state
latestGossip = Gossip()
publish(localGossip)
publish(latestGossip)
context.become(removed)
// make sure the final (removed) state is published
// before shutting down
@ -426,7 +433,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
val versionedGossip = newGossip :+ vclockNode
latestGossip = versionedGossip seen selfAddress
publish(localGossip)
publish(latestGossip)
}
/**
@ -515,7 +522,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
}
stats = stats.incrementReceivedGossipCount
publish(localGossip)
publish(latestGossip)
if (envelope.conversation &&
(conflict || (winningGossip ne remoteGossip) || (latestGossip ne remoteGossip))) {
@ -602,12 +609,18 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
if (localGossip.convergence) {
// we have convergence - so we can't have unreachable nodes
val numberOfMembers = localMembers.size
def isJoiningToUp(m: Member): Boolean = m.status == Joining && numberOfMembers >= MinNrOfMembers
// transform the node member ring
val newMembers = localMembers collect {
// 1. Move JOINING => UP (once all nodes have seen that this node is JOINING e.g. we have a convergence)
case member if member.status == Joining member copy (status = Up)
// 2. Move LEAVING => EXITING (once we have a convergence on LEAVING *and* if we have a successful partition handoff)
case member if member.status == Leaving && hasPartionHandoffCompletedSuccessfully member copy (status = Exiting)
// 1. Move JOINING => UP (once all nodes have seen that this node is JOINING, i.e. we have a convergence)
// and minimum number of nodes have joined the cluster
case member if isJoiningToUp(member) member copy (status = Up)
// 2. Move LEAVING => EXITING (once we have a convergence on LEAVING
// *and* if we have a successful partition handoff)
case member if member.status == Leaving && hasPartionHandoffCompletedSuccessfully
member copy (status = Exiting)
// 3. Everyone else that is not Exiting stays as they are
case member if member.status != Exiting member
// 4. Move EXITING => REMOVED - e.g. remove the nodes from the 'members' set/node ring and seen table
@ -621,10 +634,10 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
// Repeat the checking for transitions between JOINING -> UP, LEAVING -> EXITING, EXITING -> REMOVED
// to check for state-changes and to store away removed and exiting members for later notification
// 1. check for state-changes to update
// 2. store away removed and exiting members so we can separate the pure state changes (that can be retried on collision) and the side-effecting message sending
// 2. store away removed and exiting members so we can separate the pure state changes
val (removedMembers, newMembers1) = localMembers partition (_.status == Exiting)
val (upMembers, newMembers2) = newMembers1 partition (_.status == Joining)
val (upMembers, newMembers2) = newMembers1 partition (isJoiningToUp(_))
val exitingMembers = newMembers2 filter (_.status == Leaving && hasPartionHandoffCompletedSuccessfully)
@ -715,7 +728,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
log.info("Cluster Node [{}] - Leader is marking unreachable node [{}] as DOWN", selfAddress, member.address)
}
publish(localGossip)
publish(latestGossip)
}
}
}
@ -752,7 +765,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
log.error("Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]", selfAddress, newlyDetectedUnreachableMembers.mkString(", "))
publish(localGossip)
publish(latestGossip)
}
}
}
@ -763,14 +776,14 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
def isSingletonCluster: Boolean = latestGossip.isSingletonCluster
def isAvailable: Boolean = latestGossip.isAvailable(selfAddress)
def isAvailable: Boolean = !latestGossip.isUnreachable(selfAddress)
/**
* Gossips latest gossip to a random member in the set of members passed in as argument.
*
* @return the used [[akka.actor.Address] if any
*/
private def gossipToRandomNodeOf(addresses: IndexedSeq[Address]): Option[Address] = {
private def gossipToRandomNodeOf(addresses: immutable.IndexedSeq[Address]): Option[Address] = {
log.debug("Cluster Node [{}] - Selecting random node to gossip to [{}]", selfAddress, addresses.mkString(", "))
// filter out myself
val peer = selectRandomNode(addresses filterNot (_ == selfAddress))
@ -790,8 +803,8 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
def gossipTo(address: Address, gossipMsg: GossipEnvelope): Unit = if (address != selfAddress)
coreSender ! SendClusterMessage(address, gossipMsg)
def publish(oldGossip: Gossip): Unit = {
publisher ! PublishChanges(oldGossip, latestGossip)
def publish(newGossip: Gossip): Unit = {
publisher ! PublishChanges(newGossip)
if (PublishStatsInterval == Duration.Zero) publishInternalStats()
}
@ -823,7 +836,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
* 5. seed3 retries the join procedure and gets acks from seed2 first, and then joins to seed2
*
*/
private[cluster] final class JoinSeedNodeProcess(seedNodes: IndexedSeq[Address]) extends Actor with ActorLogging {
private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq[Address]) extends Actor with ActorLogging {
import InternalClusterAction._
def selfAddress = Cluster(context.system).selfAddress
@ -877,6 +890,42 @@ private[cluster] final class ClusterCoreSender extends Actor with ActorLogging {
}
}
/**
* INTERNAL API
*
* The supplied callback will be run, once, when current cluster member is `Up`.
*/
private[cluster] class OnMemberUpListener(callback: Runnable) extends Actor with ActorLogging {
import ClusterEvent._
val cluster = Cluster(context.system)
// subscribe to MemberUp, re-subscribe when restart
override def preStart(): Unit =
cluster.subscribe(self, classOf[MemberUp])
override def postStop(): Unit =
cluster.unsubscribe(self)
def receive = {
case state: CurrentClusterState
if (state.members.exists(isSelfUp(_)))
done()
case MemberUp(m)
if (isSelfUp(m))
done()
}
def done(): Unit = {
try callback.run() catch {
case NonFatal(e) log.error(e, "OnMemberUp callback failed with [{}]", e.getMessage)
} finally {
context stop self
}
}
def isSelfUp(m: Member): Boolean =
m.address == cluster.selfAddress && m.status == MemberStatus.Up
}
/**
* INTERNAL API
*/

View file

@ -4,12 +4,16 @@
package akka.cluster
import language.postfixOps
import scala.collection.immutable.SortedSet
import scala.collection.immutable
import scala.collection.immutable.{ VectorBuilder, SortedSet }
import akka.actor.{ Actor, ActorLogging, ActorRef, Address }
import akka.cluster.ClusterEvent._
import akka.cluster.MemberStatus._
import akka.event.EventStream
import akka.actor.AddressTerminated
import java.lang.Iterable
import akka.japi.Util.immutableSeq
import akka.util.Collections.EmptyImmutableSeq
/**
* Domain events published to the event bus.
@ -28,9 +32,8 @@ object ClusterEvent {
* Current snapshot state of the cluster. Sent to new subscriber.
*/
case class CurrentClusterState(
members: SortedSet[Member] = SortedSet.empty,
members: immutable.SortedSet[Member] = immutable.SortedSet.empty,
unreachable: Set[Member] = Set.empty,
convergence: Boolean = false,
seenBy: Set[Address] = Set.empty,
leader: Option[Address] = None) extends ClusterDomainEvent {
@ -47,19 +50,15 @@ object ClusterEvent {
* Java API
* Read only
*/
def getUnreachable: java.util.Set[Member] = {
import scala.collection.JavaConverters._
unreachable.asJava
}
def getUnreachable: java.util.Set[Member] =
scala.collection.JavaConverters.setAsJavaSetConverter(unreachable).asJava
/**
* Java API
* Read only
*/
def getSeenBy: java.util.Set[Address] = {
import scala.collection.JavaConverters._
seenBy.asJava
}
def getSeenBy: java.util.Set[Address] =
scala.collection.JavaConverters.setAsJavaSetConverter(seenBy).asJava
/**
* Java API
@ -76,57 +75,47 @@ object ClusterEvent {
}
/**
* A new member joined the cluster.
* A new member joined the cluster. Only published after convergence.
*/
case class MemberJoined(member: Member) extends MemberEvent {
if (member.status != Joining) throw new IllegalArgumentException("Expected Joining status, got: " + member)
}
/**
* Member status changed to Up
* Member status changed to Up. Only published after convergence.
*/
case class MemberUp(member: Member) extends MemberEvent {
if (member.status != Up) throw new IllegalArgumentException("Expected Up status, got: " + member)
}
/**
* Member status changed to Leaving
* Member status changed to Leaving. Only published after convergence.
*/
case class MemberLeft(member: Member) extends MemberEvent {
if (member.status != Leaving) throw new IllegalArgumentException("Expected Leaving status, got: " + member)
}
/**
* Member status changed to Exiting
* Member status changed to Exiting. Only published after convergence.
*/
case class MemberExited(member: Member) extends MemberEvent {
if (member.status != Exiting) throw new IllegalArgumentException("Expected Exiting status, got: " + member)
}
/**
* A member is considered as unreachable by the failure detector.
*/
case class MemberUnreachable(member: Member) extends MemberEvent
/**
* Member status changed to Down
* Member status changed to Down. Only published after convergence.
*/
case class MemberDowned(member: Member) extends MemberEvent {
if (member.status != Down) throw new IllegalArgumentException("Expected Down status, got: " + member)
}
/**
* Member completely removed from the cluster
* Member completely removed from the cluster. Only published after convergence.
*/
case class MemberRemoved(member: Member) extends MemberEvent {
if (member.status != Removed) throw new IllegalArgumentException("Expected Removed status, got: " + member)
}
/**
* Cluster convergence state changed.
*/
case class ConvergenceChanged(convergence: Boolean) extends ClusterDomainEvent
/**
* Leader of the cluster members changed. Only published after convergence.
*/
@ -138,12 +127,23 @@ object ClusterEvent {
def getLeader: Address = leader orNull
}
/**
* A member is considered as unreachable by the failure detector.
*/
case class UnreachableMember(member: Member) extends ClusterDomainEvent
/**
* INTERNAL API
*
* Current snapshot of cluster member metrics. Published to subscribers.
* Current snapshot of cluster node metrics. Published to subscribers.
*/
case class ClusterMetricsChanged(nodes: Set[NodeMetrics]) extends ClusterDomainEvent
case class ClusterMetricsChanged(nodeMetrics: Set[NodeMetrics]) extends ClusterDomainEvent {
/**
* Java API
*/
def getNodeMetrics: java.lang.Iterable[NodeMetrics] =
scala.collection.JavaConverters.asJavaIterableConverter(nodeMetrics).asJava
}
/**
* INTERNAL API
@ -159,29 +159,42 @@ object ClusterEvent {
/**
* INTERNAL API
*/
private[cluster] def diff(oldGossip: Gossip, newGossip: Gossip): IndexedSeq[ClusterDomainEvent] = {
val newMembers = newGossip.members -- oldGossip.members
private[cluster] def diffUnreachable(oldGossip: Gossip, newGossip: Gossip): immutable.Seq[UnreachableMember] =
if (newGossip eq oldGossip) Nil
else {
val newUnreachable = newGossip.overview.unreachable -- oldGossip.overview.unreachable
val unreachableEvents = newUnreachable map UnreachableMember
val membersGroupedByAddress = (newGossip.members.toList ++ oldGossip.members.toList).groupBy(_.address)
immutable.Seq.empty ++ unreachableEvents
}
/**
* INTERNAL API.
*/
private[cluster] def diffMemberEvents(oldGossip: Gossip, newGossip: Gossip): immutable.Seq[MemberEvent] =
if (newGossip eq oldGossip) Nil
else {
val newMembers = newGossip.members -- oldGossip.members
val membersGroupedByAddress = List(newGossip.members, oldGossip.members).flatten.groupBy(_.address)
val changedMembers = membersGroupedByAddress collect {
case (_, newMember :: oldMember :: Nil) if newMember.status != oldMember.status newMember
}
val memberEvents = (newMembers ++ changedMembers) map { m
if (m.status == Joining) MemberJoined(m)
else if (m.status == Up) MemberUp(m)
else if (m.status == Leaving) MemberLeft(m)
else if (m.status == Exiting) MemberExited(m)
else throw new IllegalStateException("Unexpected member status: " + m)
m.status match {
case Joining MemberJoined(m)
case Up MemberUp(m)
case Leaving MemberLeft(m)
case Exiting MemberExited(m)
case _ throw new IllegalStateException("Unexpected member status: " + m)
}
}
val allNewUnreachable = newGossip.overview.unreachable -- oldGossip.overview.unreachable
val (newDowned, newUnreachable) = allNewUnreachable partition { _.status == Down }
val newDowned = allNewUnreachable filter { _.status == Down }
val downedEvents = newDowned map MemberDowned
val unreachableEvents = newUnreachable map MemberUnreachable
val unreachableGroupedByAddress =
(newGossip.overview.unreachable.toList ++ oldGossip.overview.unreachable.toList).groupBy(_.address)
List(newGossip.overview.unreachable, oldGossip.overview.unreachable).flatten.groupBy(_.address)
val unreachableDownMembers = unreachableGroupedByAddress collect {
case (_, newMember :: oldMember :: Nil) if newMember.status == Down && newMember.status != oldMember.status
newMember
@ -192,23 +205,29 @@ object ClusterEvent {
MemberRemoved(m.copy(status = Removed))
}
val newConvergence = newGossip.convergence
val convergenceChanged = newConvergence != oldGossip.convergence
val convergenceEvents = if (convergenceChanged) Seq(ConvergenceChanged(newConvergence)) else Seq.empty
val leaderEvents =
if (newGossip.leader != oldGossip.leader) Seq(LeaderChanged(newGossip.leader))
else Seq.empty
val newSeenBy = newGossip.seenBy
val seenEvents =
if (convergenceChanged || newSeenBy != oldGossip.seenBy) Seq(SeenChanged(newConvergence, newSeenBy))
else Seq.empty
memberEvents.toIndexedSeq ++ unreachableEvents ++ downedEvents ++ unreachableDownedEvents ++ removedEvents ++
leaderEvents ++ convergenceEvents ++ seenEvents
(new VectorBuilder[MemberEvent]() ++= memberEvents ++= downedEvents ++= unreachableDownedEvents
++= removedEvents).result()
}
/**
* INTERNAL API
*/
private[cluster] def diffLeader(oldGossip: Gossip, newGossip: Gossip): immutable.Seq[LeaderChanged] =
if (newGossip.leader != oldGossip.leader) List(LeaderChanged(newGossip.leader))
else Nil
/**
* INTERNAL API
*/
private[cluster] def diffSeen(oldGossip: Gossip, newGossip: Gossip): immutable.Seq[SeenChanged] =
if (newGossip eq oldGossip) Nil
else {
val newConvergence = newGossip.convergence
val newSeenBy = newGossip.seenBy
if (newConvergence != oldGossip.convergence || newSeenBy != oldGossip.seenBy)
List(SeenChanged(newConvergence, newSeenBy))
else Nil
}
}
/**
@ -220,34 +239,30 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto
import InternalClusterAction._
var latestGossip: Gossip = Gossip()
// Keep track of LeaderChanged event. Should not be published until
// convergence, and it should only be published when leader actually
// changed to another node. 3 states:
// - None: No LeaderChanged detected yet, nothing published yet
// - Some(Left): Stashed LeaderChanged to be published later, when convergence
// - Some(Right): Latest published LeaderChanged
var leaderChangedState: Option[Either[LeaderChanged, LeaderChanged]] = None
var latestConvergedGossip: Gossip = Gossip()
var memberEvents: immutable.Seq[MemberEvent] = immutable.Seq.empty
def receive = {
case PublishChanges(oldGossip, newGossip) publishChanges(oldGossip, newGossip)
case PublishChanges(newGossip) publishChanges(newGossip)
case currentStats: CurrentInternalStats publishInternalStats(currentStats)
case PublishCurrentClusterState(receiver) publishCurrentClusterState(receiver)
case Subscribe(subscriber, to) subscribe(subscriber, to)
case Unsubscribe(subscriber, to) unsubscribe(subscriber, to)
case PublishEvent(event) publish(event)
case PublishDone sender ! PublishDone
case PublishStart publishStart()
case PublishDone publishDone(sender)
}
def eventStream: EventStream = context.system.eventStream
def publishCurrentClusterState(receiver: Option[ActorRef]): Unit = {
// The state is a mix of converged and latest gossip to mimic what you
// would have seen if you where listening to the events.
val state = CurrentClusterState(
members = latestGossip.members,
members = latestConvergedGossip.members,
unreachable = latestGossip.overview.unreachable,
convergence = latestGossip.convergence,
seenBy = latestGossip.seenBy,
leader = latestGossip.leader)
leader = latestConvergedGossip.leader)
receiver match {
case Some(ref) ref ! state
case None publish(state)
@ -264,47 +279,43 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto
case Some(c) eventStream.unsubscribe(subscriber, c)
}
def publishChanges(oldGossip: Gossip, newGossip: Gossip): Unit = {
def publishChanges(newGossip: Gossip): Unit = {
val oldGossip = latestGossip
// keep the latestGossip to be sent to new subscribers
latestGossip = newGossip
diff(oldGossip, newGossip) foreach { event
event match {
case x @ LeaderChanged(_) if leaderChangedState == Some(Right(x))
// skip, this leader has already been published
case x @ LeaderChanged(_) if oldGossip.convergence && newGossip.convergence
// leader changed and immediate convergence
leaderChangedState = Some(Right(x))
publish(x)
case x: LeaderChanged
// publish later, when convergence
leaderChangedState = Some(Left(x))
case ConvergenceChanged(true)
// now it's convergence, publish eventual stashed LeaderChanged event
leaderChangedState match {
case Some(Left(x))
leaderChangedState = Some(Right(x))
publish(x)
case _ // nothing stashed
}
publish(event)
case MemberUnreachable(m)
// first publish the diffUnreachable between the last two gossips
diffUnreachable(oldGossip, newGossip) foreach { event
publish(event)
// notify DeathWatch about unreachable node
publish(AddressTerminated(m.address))
case _
// all other events
publish(event)
publish(AddressTerminated(event.member.address))
}
// buffer up the MemberEvents waiting for convergence
memberEvents ++= diffMemberEvents(oldGossip, newGossip)
// if we have convergence then publish the MemberEvents and possibly a LeaderChanged
if (newGossip.convergence) {
val previousConvergedGossip = latestConvergedGossip
latestConvergedGossip = newGossip
memberEvents foreach publish
memberEvents = immutable.Seq.empty
diffLeader(previousConvergedGossip, latestConvergedGossip) foreach publish
}
// publish internal SeenState for testing purposes
diffSeen(oldGossip, newGossip) foreach publish
}
def publishInternalStats(currentStats: CurrentInternalStats): Unit = publish(currentStats)
def publish(event: AnyRef): Unit = eventStream publish event
def publishStart(): Unit = clearState()
def publishDone(receiver: ActorRef): Unit = {
clearState()
receiver ! PublishDoneFinished
}
def clearState(): Unit = {
latestGossip = Gossip()
latestConvergedGossip = Gossip()
}
}

View file

@ -5,7 +5,7 @@ package akka.cluster
import language.postfixOps
import scala.collection.immutable.SortedSet
import scala.collection.immutable
import scala.annotation.tailrec
import scala.concurrent.duration._
import java.net.URLEncoder
@ -96,7 +96,10 @@ private[cluster] final class ClusterHeartbeatSender extends Actor with ActorLogg
val heartbeatTask = scheduler.schedule(PeriodicTasksInitialDelay max HeartbeatInterval,
HeartbeatInterval, self, HeartbeatTick)
override def preStart(): Unit = cluster.subscribe(self, classOf[MemberEvent])
override def preStart(): Unit = {
cluster.subscribe(self, classOf[MemberEvent])
cluster.subscribe(self, classOf[UnreachableMember])
}
override def postStop(): Unit = {
heartbeatTask.cancel()
@ -112,7 +115,7 @@ private[cluster] final class ClusterHeartbeatSender extends Actor with ActorLogg
def receive = {
case HeartbeatTick heartbeat()
case s: CurrentClusterState reset(s)
case MemberUnreachable(m) removeMember(m)
case UnreachableMember(m) removeMember(m)
case MemberRemoved(m) removeMember(m)
case e: MemberEvent addMember(e.member)
case JoinInProgress(a, d) addJoinInProgress(a, d)

View file

@ -16,17 +16,70 @@ import javax.management.InstanceNotFoundException
* Interface for the cluster JMX MBean.
*/
trait ClusterNodeMBean {
/**
* Member status for this node.
*/
def getMemberStatus: String
/**
* Comma separated addresses of member nodes, sorted in the cluster ring order.
* The address format is `akka://actor-system-name@hostname:port`
*/
def getMembers: String
/**
* Comma separated addresses of unreachable member nodes.
* The address format is `akka://actor-system-name@hostname:port`
*/
def getUnreachable: String
/*
* String that will list all nodes in the node ring as follows:
* {{{
* Members:
* Member(address = akka://system0@localhost:5550, status = Up)
* Member(address = akka://system1@localhost:5551, status = Up)
* Unreachable:
* Member(address = akka://system2@localhost:5553, status = Down)
* }}}
*/
def getClusterStatus: String
/**
* Get the address of the current leader.
* The address format is `akka://actor-system-name@hostname:port`
*/
def getLeader: String
/**
* Does the cluster consist of only one member?
*/
def isSingleton: Boolean
def isConvergence: Boolean
def isAvailable: Boolean
def isRunning: Boolean
/**
* Returns true if the node is not unreachable and not `Down`
* and not `Removed`.
*/
def isAvailable: Boolean
/**
* Try to join this cluster node with the node specified by 'address'.
* The address format is `akka://actor-system-name@hostname:port`.
* A 'Join(thisNodeAddress)' command is sent to the node to join.
*/
def join(address: String)
/**
* Send command to issue state transition to LEAVING for the node specified by 'address'.
* The address format is `akka://actor-system-name@hostname:port`
*/
def leave(address: String)
/**
* Send command to DOWN the node specified by 'address'.
* The address format is `akka://actor-system-name@hostname:port`
*/
def down(address: String)
}
@ -47,34 +100,26 @@ private[akka] class ClusterJmx(cluster: Cluster, log: LoggingAdapter) {
// JMX attributes (bean-style)
/*
* Sends a string to the JMX client that will list all nodes in the node ring as follows:
* {{{
* Members:
* Member(address = akka://system0@localhost:5550, status = Up)
* Member(address = akka://system1@localhost:5551, status = Up)
* Unreachable:
* Member(address = akka://system2@localhost:5553, status = Down)
* }}}
*/
def getClusterStatus: String = {
val unreachable = clusterView.unreachableMembers
"\nMembers:\n\t" + clusterView.members.mkString("\n\t") +
{ if (unreachable.nonEmpty) "\nUnreachable:\n\t" + unreachable.mkString("\n\t") else "" }
}
def getMembers: String =
clusterView.members.toSeq.map(_.address).mkString(",")
def getUnreachable: String =
clusterView.unreachableMembers.map(_.address).mkString(",")
def getMemberStatus: String = clusterView.status.toString
def getLeader: String = clusterView.leader.toString
def getLeader: String = clusterView.leader.fold("")(_.toString)
def isSingleton: Boolean = clusterView.isSingletonCluster
def isConvergence: Boolean = clusterView.convergence
def isAvailable: Boolean = clusterView.isAvailable
def isRunning: Boolean = clusterView.isRunning
// JMX commands
def join(address: String) = cluster.join(AddressFromURIString(address))

View file

@ -5,7 +5,7 @@
package akka.cluster
import java.io.Closeable
import scala.collection.immutable.SortedSet
import scala.collection.immutable
import akka.actor.{ Actor, ActorRef, ActorSystemImpl, Address, Props }
import akka.cluster.ClusterEvent._
import akka.actor.PoisonPill
@ -45,11 +45,12 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable {
override def postStop(): Unit = cluster.unsubscribe(self)
def receive = {
case e: ClusterDomainEvent e match {
case SeenChanged(convergence, seenBy)
state = state.copy(convergence = convergence, seenBy = seenBy)
state = state.copy(seenBy = seenBy)
case MemberRemoved(member)
state = state.copy(members = state.members - member, unreachable = state.unreachable - member)
case MemberUnreachable(member)
case UnreachableMember(member)
// replace current member with new member (might have different status, only address is used in equals)
state = state.copy(members = state.members - member, unreachable = state.unreachable - member + member)
case MemberDowned(member)
@ -59,11 +60,10 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable {
// replace current member with new member (might have different status, only address is used in equals)
state = state.copy(members = state.members - event.member + event.member)
case LeaderChanged(leader) state = state.copy(leader = leader)
case ConvergenceChanged(convergence) state = state.copy(convergence = convergence)
case s: CurrentClusterState state = s
case CurrentInternalStats(stats) _latestStats = stats
case ClusterMetricsChanged(nodes) _clusterMetrics = nodes
case _ // ignore, not interesting
}
}
}).withDispatcher(cluster.settings.UseDispatcher), name = "clusterEventBusListener")
}
@ -74,14 +74,14 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable {
}
/**
* Returns true if the cluster node is up and running, false if it is shut down.
* Returns true if this cluster instance has be shutdown.
*/
def isRunning: Boolean = cluster.isRunning
def isTerminated: Boolean = cluster.isTerminated
/**
* Current cluster members, sorted by address.
*/
def members: SortedSet[Member] = state.members
def members: immutable.SortedSet[Member] = state.members
/**
* Members that has been detected as unreachable.
@ -108,21 +108,19 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable {
def leader: Option[Address] = state.leader
/**
* Is this node a singleton cluster?
* Does the cluster consist of only one member?
*/
def isSingletonCluster: Boolean = members.size == 1
/**
* Checks if we have a cluster convergence.
*/
def convergence: Boolean = state.convergence
/**
* Returns true if the node is UP or JOINING.
* Returns true if the node is not unreachable and not `Down`
* and not `Removed`.
*/
def isAvailable: Boolean = {
val myself = self
!unreachableMembers.contains(myself) && !myself.status.isUnavailable
!unreachableMembers.contains(myself) &&
myself.status != MemberStatus.Down &&
myself.status != MemberStatus.Removed
}
/**

View file

@ -3,15 +3,16 @@
*/
package akka.cluster
import scala.collection.immutable
import com.typesafe.config.Config
import scala.concurrent.duration.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.ConfigurationException
import scala.collection.JavaConverters._
import akka.actor.Address
import akka.actor.AddressFromURIString
import akka.dispatch.Dispatchers
import scala.concurrent.duration.FiniteDuration
import akka.japi.Util.immutableSeq
class ClusterSettings(val config: Config, val systemName: String) {
import config._
@ -45,7 +46,8 @@ class ClusterSettings(val config: Config, val systemName: String) {
require(n > 0, "failure-detector.monitored-by-nr-of-members must be > 0"); n
}
final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map { case AddressFromURIString(addr) addr }.toIndexedSeq
final val SeedNodes: immutable.IndexedSeq[Address] =
immutableSeq(getStringList("akka.cluster.seed-nodes")).map { case AddressFromURIString(addr) addr }.toVector
final val SeedNodeTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS)
final val PeriodicTasksInitialDelay: FiniteDuration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
@ -54,6 +56,10 @@ class ClusterSettings(val config: Config, val systemName: String) {
final val PublishStatsInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.publish-stats-interval"), MILLISECONDS)
final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join")
final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down")
final val MinNrOfMembers: Int = {
val n = getInt("akka.cluster.min-nr-of-members")
require(n > 0, "min-nr-of-members must be > 0"); n
}
final val JmxEnabled: Boolean = getBoolean("akka.cluster.jmx.enabled")
final val JoinTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS)
final val UseDispatcher: String = getString("akka.cluster.use-dispatcher") match {
@ -69,9 +75,16 @@ class ClusterSettings(val config: Config, val systemName: String) {
callTimeout = Duration(getMilliseconds("akka.cluster.send-circuit-breaker.call-timeout"), MILLISECONDS),
resetTimeout = Duration(getMilliseconds("akka.cluster.send-circuit-breaker.reset-timeout"), MILLISECONDS))
final val MetricsEnabled: Boolean = getBoolean("akka.cluster.metrics.enabled")
final val MetricsInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.metrics.metrics-interval"), MILLISECONDS)
final val MetricsCollectorClass: String = getString("akka.cluster.metrics.collector-class")
final val MetricsInterval: FiniteDuration = {
val d = Duration(getMilliseconds("akka.cluster.metrics.collect-interval"), MILLISECONDS)
require(d > Duration.Zero, "metrics.collect-interval must be > 0"); d
}
final val MetricsGossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.metrics.gossip-interval"), MILLISECONDS)
final val MetricsRateOfDecay: Int = getInt("akka.cluster.metrics.rate-of-decay")
final val MetricsMovingAverageHalfLife: FiniteDuration = {
val d = Duration(getMilliseconds("akka.cluster.metrics.moving-average-half-life"), MILLISECONDS)
require(d > Duration.Zero, "metrics.moving-average-half-life must be > 0"); d
}
}
case class CircuitBreakerSettings(maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration)

View file

@ -5,14 +5,14 @@
package akka.cluster
import akka.actor.Address
import scala.collection.immutable.SortedSet
import scala.collection.immutable
import MemberStatus._
/**
* Internal API
*/
private[cluster] object Gossip {
val emptyMembers: SortedSet[Member] = SortedSet.empty
val emptyMembers: immutable.SortedSet[Member] = immutable.SortedSet.empty
}
/**
@ -50,7 +50,7 @@ private[cluster] object Gossip {
*/
private[cluster] case class Gossip(
overview: GossipOverview = GossipOverview(),
members: SortedSet[Member] = Gossip.emptyMembers, // sorted set of members with their status, sorted by address
members: immutable.SortedSet[Member] = Gossip.emptyMembers, // sorted set of members with their status, sorted by address
version: VectorClock = VectorClock()) // vector clock version
extends ClusterMessage // is a serializable cluster message
with Versioned[Gossip] {
@ -135,7 +135,7 @@ private[cluster] case class Gossip(
* Checks if we have a cluster convergence. If there are any unreachable nodes then we can't have a convergence -
* waiting for user to act (issuing DOWN) or leader to act (issuing DOWN through auto-down).
*
* @return Some(convergedGossip) if convergence have been reached and None if not
* @return true if convergence have been reached and false if not
*/
def convergence: Boolean = {
val unreachable = overview.unreachable
@ -151,8 +151,10 @@ private[cluster] case class Gossip(
def allMembersInSeen = members.forall(m seen.contains(m.address))
def seenSame: Boolean =
if (seen.isEmpty) false
else {
if (seen.isEmpty) {
// if both seen and members are empty, then every(no)body has seen the same thing
members.isEmpty
} else {
val values = seen.values
val seenHead = values.head
values.forall(_ == seenHead)
@ -168,15 +170,10 @@ private[cluster] case class Gossip(
def isSingletonCluster: Boolean = members.size == 1
/**
* Returns true if the node is UP or JOINING.
* Returns true if the node is in the unreachable set
*/
def isAvailable(address: Address): Boolean = !isUnavailable(address)
def isUnavailable(address: Address): Boolean = {
val isUnreachable = overview.unreachable exists { _.address == address }
val hasUnavailableMemberStatus = members exists { m m.status.isUnavailable && m.address == address }
isUnreachable || hasUnavailableMemberStatus
}
def isUnreachable(address: Address): Boolean =
overview.unreachable exists { _.address == address }
def member(address: Address): Member = {
members.find(_.address == address).orElse(overview.unreachable.find(_.address == address)).

View file

@ -6,7 +6,7 @@ package akka.cluster
import language.implicitConversions
import scala.collection.immutable.SortedSet
import scala.collection.immutable
import scala.collection.GenTraversableOnce
import akka.actor.Address
import MemberStatus._
@ -87,13 +87,7 @@ object Member {
*
* Can be one of: Joining, Up, Leaving, Exiting and Down.
*/
abstract class MemberStatus extends ClusterMessage {
/**
* Using the same notion for 'unavailable' as 'non-convergence': DOWN
*/
def isUnavailable: Boolean = this == Down
}
abstract class MemberStatus extends ClusterMessage
object MemberStatus {
case object Joining extends MemberStatus

View file

@ -160,7 +160,7 @@ case class VectorClock(
* Compare two vector clocks. The outcomes will be one of the following:
* <p/>
* {{{
* 1. Clock 1 is BEFORE (>) Clock 2 if there exists an i such that c1(i) <= c(2) and there does not exist a j such that c1(j) > c2(j).
* 1. Clock 1 is BEFORE (>) Clock 2 if there exists an i such that c1(i) <= c2(i) and there does not exist a j such that c1(j) > c2(j).
* 2. Clock 1 is CONCURRENT (<>) to Clock 2 if there exists an i, j such that c1(i) < c2(i) and c1(j) > c2(j).
* 3. Clock 1 is AFTER (<) Clock 2 otherwise.
* }}}

View file

@ -0,0 +1,434 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.routing
import java.util.Arrays
import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.collection.immutable
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
import akka.actor.OneForOneStrategy
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.dispatch.Dispatchers
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.ClusterMetricsChanged
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.cluster.NodeMetrics
import akka.cluster.StandardMetrics.Cpu
import akka.cluster.StandardMetrics.HeapMemory
import akka.event.Logging
import akka.japi.Util.immutableSeq
import akka.routing.Broadcast
import akka.routing.Destination
import akka.routing.FromConfig
import akka.routing.NoRouter
import akka.routing.Resizer
import akka.routing.Route
import akka.routing.RouteeProvider
import akka.routing.RouterConfig
object AdaptiveLoadBalancingRouter {
private val escalateStrategy: SupervisorStrategy = OneForOneStrategy() {
case _ SupervisorStrategy.Escalate
}
}
/**
* A Router that performs load balancing of messages to cluster nodes based on
* cluster metric data.
*
* It uses random selection of routees based probabilities derived from
* the remaining capacity of corresponding node.
*
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical
* sense as this means that the router should both create new actors and use the 'routees'
* actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
* be ignored if the router is defined in the configuration file for the actor being used.
*
* <h1>Supervision Setup</h1>
*
* The router creates a head actor which supervises and/or monitors the
* routees. Instances are created as children of this actor, hence the
* children are not supervised by the parent of the router. Common choices are
* to always escalate (meaning that fault handling is always applied to all
* children simultaneously; this is the default) or use the parents strategy,
* which will result in routed children being treated individually, but it is
* possible as well to use Routers to give different supervisor strategies to
* different groups of children.
*
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
* @param routees string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
@SerialVersionUID(1L)
case class AdaptiveLoadBalancingRouter(
metricsSelector: MetricsSelector = MixMetricsSelector,
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil,
override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = AdaptiveLoadBalancingRouter.escalateStrategy)
extends RouterConfig with AdaptiveLoadBalancingRouterLike {
/**
* Constructor that sets nrOfInstances to be created.
* Java API
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
* @param nr number of routees to create
*/
def this(selector: MetricsSelector, nr: Int) = this(metricsSelector = selector, nrOfInstances = nr)
/**
* Constructor that sets the routees to be used.
* Java API
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
* @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) =
this(metricsSelector = selector, routees = immutableSeq(routeePaths))
/**
* Constructor that sets the resizer to be used.
* Java API
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
*/
def this(selector: MetricsSelector, resizer: Resizer) =
this(metricsSelector = selector, resizer = Some(resizer))
/**
* Java API for setting routerDispatcher
*/
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingRouter =
copy(routerDispatcher = dispatcherId)
/**
* Java API for setting the supervisor strategy to be used for the head
* Router actor.
*/
def withSupervisorStrategy(strategy: SupervisorStrategy): AdaptiveLoadBalancingRouter =
copy(supervisorStrategy = strategy)
/**
* Uses the resizer of the given RouterConfig if this RouterConfig
* doesn't have one, i.e. the resizer defined in code is used if
* resizer was not defined in config.
*/
override def withFallback(other: RouterConfig): RouterConfig = other match {
case _: FromConfig | _: NoRouter this
case otherRouter: AdaptiveLoadBalancingRouter
val useResizer =
if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer
else this.resizer
copy(resizer = useResizer)
case _ throw new IllegalArgumentException("Expected AdaptiveLoadBalancingRouter, got [%s]".format(other))
}
}
/**
* INTERNAL API.
*
* This strategy is a metrics-aware router which performs load balancing of messages to
* cluster nodes based on cluster metric data. It consumes [[akka.cluster.ClusterMetricsChanged]]
* events and the [[akka.cluster.routing.MetricsSelector]] creates an mix of
* weighted routees based on the node metrics. Messages are routed randomly to the
* weighted routees, i.e. nodes with lower load are more likely to be used than nodes with
* higher load
*/
trait AdaptiveLoadBalancingRouterLike { this: RouterConfig
def metricsSelector: MetricsSelector
def nrOfInstances: Int
def routees: immutable.Iterable[String]
def routerDispatcher: String
override def createRoute(routeeProvider: RouteeProvider): Route = {
if (resizer.isEmpty) {
if (routees.isEmpty) routeeProvider.createRoutees(nrOfInstances)
else routeeProvider.registerRouteesFor(routees)
}
val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
// The current weighted routees, if any. Weights are produced by the metricsSelector
// via the metricsListener Actor. It's only updated by the actor, but accessed from
// the threads of the senders.
@volatile var weightedRoutees: Option[WeightedRoutees] = None
// subscribe to ClusterMetricsChanged and update weightedRoutees
val metricsListener = routeeProvider.context.actorOf(Props(new Actor {
val cluster = Cluster(context.system)
override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterMetricsChanged])
override def postStop(): Unit = cluster.unsubscribe(self)
def receive = {
case ClusterMetricsChanged(metrics) receiveMetrics(metrics)
case _: CurrentClusterState // ignore
}
def receiveMetrics(metrics: Set[NodeMetrics]): Unit = {
// this is the only place from where weightedRoutees is updated
weightedRoutees = Some(new WeightedRoutees(routeeProvider.routees, cluster.selfAddress,
metricsSelector.weights(metrics)))
}
}).withDispatcher(routerDispatcher), name = "metricsListener")
def getNext(): ActorRef = weightedRoutees match {
case Some(weighted)
if (weighted.isEmpty) routeeProvider.context.system.deadLetters
else weighted(ThreadLocalRandom.current.nextInt(weighted.total) + 1)
case None
val currentRoutees = routeeProvider.routees
if (currentRoutees.isEmpty) routeeProvider.context.system.deadLetters
else currentRoutees(ThreadLocalRandom.current.nextInt(currentRoutees.size))
}
{
case (sender, message)
message match {
case Broadcast(msg) toAll(sender, routeeProvider.routees)
case msg List(Destination(sender, getNext()))
}
}
}
}
/**
* MetricsSelector that uses the heap metrics.
* Low heap capacity => small weight.
*/
@SerialVersionUID(1L)
case object HeapMetricsSelector extends CapacityMetricsSelector {
/**
* Java API: get the singleton instance
*/
def getInstance = this
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case HeapMemory(address, _, used, committed, max)
val capacity = max match {
case None (committed - used).toDouble / committed
case Some(m) (m - used).toDouble / m
}
(address, capacity)
}.toMap
}
}
/**
* MetricsSelector that uses the combined CPU metrics.
* Combined CPU is sum of User + Sys + Nice + Wait, in percentage.
* Low cpu capacity => small weight.
*/
@SerialVersionUID(1L)
case object CpuMetricsSelector extends CapacityMetricsSelector {
/**
* Java API: get the singleton instance
*/
def getInstance = this
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case Cpu(address, _, _, Some(cpuCombined), _)
val capacity = 1.0 - cpuCombined
(address, capacity)
}.toMap
}
}
/**
* MetricsSelector that uses the system load average metrics.
* System load average is OS-specific average load on the CPUs in the system,
* for the past 1 minute. The system is possibly nearing a bottleneck if the
* system load average is nearing number of cpus/cores.
* Low load average capacity => small weight.
*/
@SerialVersionUID(1L)
case object SystemLoadAverageMetricsSelector extends CapacityMetricsSelector {
/**
* Java API: get the singleton instance
*/
def getInstance = this
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case Cpu(address, _, Some(systemLoadAverage), _, processors)
val capacity = 1.0 - math.min(1.0, systemLoadAverage / processors)
(address, capacity)
}.toMap
}
}
/**
* Singleton instance of the default MixMetricsSelector, which uses [akka.cluster.routing.HeapMetricsSelector],
* [akka.cluster.routing.CpuMetricsSelector], and [akka.cluster.routing.SystemLoadAverageMetricsSelector]
*/
@SerialVersionUID(1L)
object MixMetricsSelector extends MixMetricsSelectorBase(
Vector(HeapMetricsSelector, CpuMetricsSelector, SystemLoadAverageMetricsSelector)) {
/**
* Java API: get the default singleton instance
*/
def getInstance = this
}
/**
* MetricsSelector that combines other selectors and aggregates their capacity
* values. By default it uses [akka.cluster.routing.HeapMetricsSelector],
* [akka.cluster.routing.CpuMetricsSelector], and [akka.cluster.routing.SystemLoadAverageMetricsSelector]
*/
@SerialVersionUID(1L)
case class MixMetricsSelector(
selectors: immutable.IndexedSeq[CapacityMetricsSelector])
extends MixMetricsSelectorBase(selectors)
/**
* Base class for MetricsSelector that combines other selectors and aggregates their capacity.
*/
@SerialVersionUID(1L)
abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMetricsSelector])
extends CapacityMetricsSelector {
/**
* Java API
*/
def this(selectors: java.lang.Iterable[CapacityMetricsSelector]) = this(immutableSeq(selectors).toVector)
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
// aggregated average of the capacities by address
combined.foldLeft(Map.empty[Address, (Double, Int)].withDefaultValue((0.0, 0))) {
case (acc, (address, capacity))
val (sum, count) = acc(address)
acc + (address -> (sum + capacity, count + 1))
}.map {
case (addr, (sum, count)) (addr -> sum / count)
}
}
}
/**
* A MetricsSelector is responsible for producing weights from the node metrics.
*/
@SerialVersionUID(1L)
trait MetricsSelector extends Serializable {
/**
* The weights per address, based on the the nodeMetrics.
*/
def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int]
}
/**
* A MetricsSelector producing weights from remaining capacity.
* The weights are typically proportional to the remaining capacity.
*/
abstract class CapacityMetricsSelector extends MetricsSelector {
/**
* Remaining capacity for each node. The value is between
* 0.0 and 1.0, where 0.0 means no remaining capacity (full
* utilization) and 1.0 means full remaining capacity (zero
* utilization).
*/
def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double]
/**
* Converts the capacity values to weights. The node with lowest
* capacity gets weight 1 (lowest usable capacity is 1%) and other
* nodes gets weights proportional to their capacity compared to
* the node with lowest capacity.
*/
def weights(capacity: Map[Address, Double]): Map[Address, Int] = {
if (capacity.isEmpty) Map.empty[Address, Int]
else {
val (_, min) = capacity.minBy { case (_, c) c }
// lowest usable capacity is 1% (>= 0.5% will be rounded to weight 1), also avoids div by zero
val divisor = math.max(0.01, min)
capacity map { case (addr, c) (addr -> math.round((c) / divisor).toInt) }
}
}
/**
* The weights per address, based on the capacity produced by
* the nodeMetrics.
*/
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] =
weights(capacity(nodeMetrics))
}
/**
* INTERNAL API
*
* Pick routee based on its weight. Higher weight, higher probability.
*/
private[cluster] class WeightedRoutees(refs: immutable.IndexedSeq[ActorRef], selfAddress: Address, weights: Map[Address, Int]) {
// fill an array of same size as the refs with accumulated weights,
// binarySearch is used to pick the right bucket from a requested value
// from 1 to the total sum of the used weights.
private val buckets: Array[Int] = {
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
case Address(_, _, None, None) selfAddress
case a a
}
val buckets = Array.ofDim[Int](refs.size)
val meanWeight = if (weights.isEmpty) 1 else weights.values.sum / weights.size
val w = weights.withDefaultValue(meanWeight) // we dont necessarily have metrics for all addresses
var i = 0
var sum = 0
refs foreach { ref
sum += w(fullAddress(ref))
buckets(i) = sum
i += 1
}
buckets
}
def isEmpty: Boolean = buckets.length == 0
def total: Int = {
require(!isEmpty, "WeightedRoutees must not be used when empty")
buckets(buckets.length - 1)
}
/**
* Pick the routee matching a value, from 1 to total.
*/
def apply(value: Int): ActorRef = {
require(1 <= value && value <= total, "value must be between [1 - %s]" format total)
refs(idx(Arrays.binarySearch(buckets, value)))
}
/**
* Converts the result of Arrays.binarySearch into a index in the buckets array
* see documentation of Arrays.binarySearch for what it returns
*/
private def idx(i: Int): Int = {
if (i >= 0) i // exact match
else {
val j = math.abs(i + 1)
if (j >= buckets.length) throw new IndexOutOfBoundsException(
"Requested index [%s] is > max index [%s]".format(i, buckets.length))
else j
}
}
}

View file

@ -127,7 +127,7 @@ case class ClusterRouterSettings private[akka] (
if (isRouteesPathDefined && maxInstancesPerNode != 1)
throw new IllegalArgumentException("maxInstancesPerNode of cluster router must be 1 when routeesPath is defined")
val routeesPathElements: Iterable[String] = routeesPath match {
val routeesPathElements: immutable.Iterable[String] = routeesPath match {
case RelativeActorPath(elements) elements
case _
throw new IllegalArgumentException("routeesPath [%s] is not a valid relative actor path" format routeesPath)
@ -248,9 +248,11 @@ private[akka] class ClusterRouteeProvider(
*/
private[akka] class ClusterRouterActor extends Router {
// subscribe to cluster changes, MemberEvent
// re-subscribe when restart
override def preStart(): Unit = cluster.subscribe(self, classOf[MemberEvent])
override def preStart(): Unit = {
cluster.subscribe(self, classOf[MemberEvent])
cluster.subscribe(self, classOf[UnreachableMember])
}
override def postStop(): Unit = cluster.unsubscribe(self)
// lazy to not interfere with RoutedActorCell initialization
@ -264,6 +266,19 @@ private[akka] class ClusterRouterActor extends Router {
def fullAddress(actorRef: ActorRef): Address = routeeProvider.fullAddress(actorRef)
def unregisterRoutees(member: Member) = {
val address = member.address
routeeProvider.nodes -= address
// unregister routees that live on that node
val affectedRoutes = routeeProvider.routees.filter(fullAddress(_) == address)
routeeProvider.unregisterRoutees(affectedRoutes)
// createRoutees will not create more than createRoutees and maxInstancesPerNode
// this is useful when totalInstances < upNodes.size
routeeProvider.createRoutees()
}
override def routerReceive: Receive = {
case s: CurrentClusterState
import Member.addressOrdering
@ -278,17 +293,10 @@ private[akka] class ClusterRouterActor extends Router {
case other: MemberEvent
// other events means that it is no longer interesting, such as
// MemberJoined, MemberLeft, MemberExited, MemberUnreachable, MemberRemoved
val address = other.member.address
routeeProvider.nodes -= address
// unregister routees that live on that node
val affectedRoutes = routeeProvider.routees.filter(fullAddress(_) == address)
routeeProvider.unregisterRoutees(affectedRoutes)
// createRoutees will not create more than createRoutees and maxInstancesPerNode
// this is useful when totalInstances < upNodes.size
routeeProvider.createRoutees()
// MemberJoined, MemberLeft, MemberExited, MemberRemoved
unregisterRoutees(other.member)
case UnreachableMember(m)
unregisterRoutees(m)
}
}

View file

@ -3,6 +3,8 @@
*/
package akka.cluster
import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
@ -54,7 +56,7 @@ abstract class ClusterDeathWatchSpec
}
"An actor watching a remote actor in the cluster" must {
"receive Terminated when watched node becomes unreachable" taggedAs LongRunningTest in {
"receive Terminated when watched node becomes Down" taggedAs LongRunningTest in within(20 seconds) {
awaitClusterUp(roles: _*)
enterBarrier("cluster-up")
@ -76,10 +78,12 @@ abstract class ClusterDeathWatchSpec
watchEstablished.await
enterBarrier("watch-established")
expectMsg(path2)
expectNoMsg
expectNoMsg(2 seconds)
enterBarrier("second-terminated")
markNodeAsUnavailable(third)
awaitCond(clusterView.unreachableMembers.exists(_.address == address(third)))
cluster.down(third)
expectMsg(path3)
enterBarrier("third-terminated")
@ -91,6 +95,8 @@ abstract class ClusterDeathWatchSpec
enterBarrier("watch-established")
runOn(third) {
markNodeAsUnavailable(second)
awaitCond(clusterView.unreachableMembers.exists(_.address == address(second)))
cluster.down(second)
}
enterBarrier("second-terminated")
enterBarrier("third-terminated")
@ -132,7 +138,7 @@ abstract class ClusterDeathWatchSpec
enterBarrier("after-3")
}
"be able to shutdown system when using remote deployed actor on node that crash" taggedAs LongRunningTest in {
"be able to shutdown system when using remote deployed actor on node that crash" taggedAs LongRunningTest in within(20 seconds) {
runOn(fourth) {
val hello = system.actorOf(Props[Hello], "hello")
hello.isInstanceOf[RemoteActorRef] must be(true)
@ -141,6 +147,9 @@ abstract class ClusterDeathWatchSpec
enterBarrier("hello-deployed")
markNodeAsUnavailable(first)
awaitCond(clusterView.unreachableMembers.exists(_.address == address(first)))
cluster.down(first)
val t = expectMsgType[Terminated]
t.actor must be(hello)

View file

@ -1,31 +0,0 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.language.postfixOps
import scala.concurrent.duration._
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
import com.typesafe.config.ConfigFactory
import akka.testkit.LongRunningTest
object ClusterMetricsDataStreamingOffMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(ConfigFactory.parseString("akka.cluster.metrics.rate-of-decay = 0")
.withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
}
class ClusterMetricsDataStreamingOffMultiJvmNode1 extends ClusterMetricsDataStreamingOffSpec
class ClusterMetricsDataStreamingOffMultiJvmNode2 extends ClusterMetricsDataStreamingOffSpec
abstract class ClusterMetricsDataStreamingOffSpec extends MultiNodeSpec(ClusterMetricsDataStreamingOffMultiJvmSpec) with MultiNodeClusterSpec with MetricSpec {
"Cluster metrics" must {
"not collect stream metric data" taggedAs LongRunningTest in within(30 seconds) {
awaitClusterUp(roles: _*)
awaitCond(clusterView.clusterMetrics.size == roles.size)
awaitCond(clusterView.clusterMetrics.flatMap(_.metrics).filter(_.trendable).forall(_.average.isEmpty))
enterBarrier("after")
}
}
}

View file

@ -28,9 +28,11 @@ class ClusterMetricsMultiJvmNode3 extends ClusterMetricsSpec
class ClusterMetricsMultiJvmNode4 extends ClusterMetricsSpec
class ClusterMetricsMultiJvmNode5 extends ClusterMetricsSpec
abstract class ClusterMetricsSpec extends MultiNodeSpec(ClusterMetricsMultiJvmSpec) with MultiNodeClusterSpec with MetricSpec {
abstract class ClusterMetricsSpec extends MultiNodeSpec(ClusterMetricsMultiJvmSpec) with MultiNodeClusterSpec {
import ClusterMetricsMultiJvmSpec._
def isSigar(collector: MetricsCollector): Boolean = collector.isInstanceOf[SigarMetricsCollector]
"Cluster metrics" must {
"periodically collect metrics on each node, publish ClusterMetricsChanged to the event stream, " +
"and gossip metrics around the node ring" taggedAs LongRunningTest in within(60 seconds) {
@ -38,9 +40,8 @@ abstract class ClusterMetricsSpec extends MultiNodeSpec(ClusterMetricsMultiJvmSp
enterBarrier("cluster-started")
awaitCond(clusterView.members.filter(_.status == MemberStatus.Up).size == roles.size)
awaitCond(clusterView.clusterMetrics.size == roles.size)
assertInitialized(cluster.settings.MetricsRateOfDecay, collectNodeMetrics(clusterView.clusterMetrics).toSet)
val collector = MetricsCollector(cluster.selfAddress, log, system.asInstanceOf[ExtendedActorSystem].dynamicAccess)
clusterView.clusterMetrics.foreach(n assertExpectedSampleSize(collector.isSigar, cluster.settings.MetricsRateOfDecay, n))
val collector = MetricsCollector(cluster.system, cluster.settings)
collector.sample.metrics.size must be > (3)
enterBarrier("after")
}
"reflect the correct number of node metrics in cluster view" taggedAs LongRunningTest in within(30 seconds) {

View file

@ -76,8 +76,6 @@ abstract class ConvergenceSpec(multiNodeConfig: ConvergenceMultiNodeConfig)
// still one unreachable
clusterView.unreachableMembers.size must be(1)
clusterView.unreachableMembers.head.address must be(thirdAddress)
// and therefore no convergence
clusterView.convergence must be(false)
}
}
@ -94,23 +92,33 @@ abstract class ConvergenceSpec(multiNodeConfig: ConvergenceMultiNodeConfig)
def memberStatus(address: Address): Option[MemberStatus] =
clusterView.members.collectFirst { case m if m.address == address m.status }
def assertNotMovedUp: Unit = {
def assertNotMovedUp(joining: Boolean): Unit = {
within(20 seconds) {
awaitCond(clusterView.members.size == 3)
if (joining) awaitCond(clusterView.members.size == 0)
else awaitCond(clusterView.members.size == 2)
awaitSeenSameState(first, second, fourth)
memberStatus(first) must be(Some(MemberStatus.Up))
memberStatus(second) must be(Some(MemberStatus.Up))
if (joining) memberStatus(first) must be(None)
else memberStatus(first) must be(Some(MemberStatus.Up))
if (joining) memberStatus(second) must be(None)
else memberStatus(second) must be(Some(MemberStatus.Up))
// leader is not allowed to move the new node to Up
memberStatus(fourth) must be(Some(MemberStatus.Joining))
// still no convergence
clusterView.convergence must be(false)
memberStatus(fourth) must be(None)
}
}
runOn(first, second, fourth) {
enterBarrier("after-join")
runOn(first, second) {
for (n 1 to 5) {
log.debug("assertNotMovedUp#" + n)
assertNotMovedUp
assertNotMovedUp(joining = false)
// wait and then check again
Thread.sleep(1.second.dilated.toMillis)
}
}
runOn(fourth) {
for (n 1 to 5) {
assertNotMovedUp(joining = true)
// wait and then check again
Thread.sleep(1.second.dilated.toMillis)
}

View file

@ -3,6 +3,7 @@
*/
package akka.cluster
import scala.collection.immutable
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import akka.remote.testkit.MultiNodeConfig
@ -35,7 +36,7 @@ abstract class JoinSeedNodeSpec
import JoinSeedNodeMultiJvmSpec._
def seedNodes: IndexedSeq[Address] = IndexedSeq(seed1, seed2, seed3)
def seedNodes: immutable.IndexedSeq[Address] = Vector(seed1, seed2, seed3)
"A cluster with seed nodes" must {
"be able to start the seed nodes concurrently" taggedAs LongRunningTest in {

View file

@ -177,7 +177,6 @@ abstract class LargeClusterSpec
Await.ready(latch, remaining)
awaitCond(clusterNodes.forall(_.readView.convergence))
val counts = clusterNodes.map(gossipCount(_))
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / clusterNodes.size, counts.min, counts.max)
log.info("Convergence of [{}] nodes reached, it took [{}], received [{}] gossip messages per node",
@ -274,7 +273,7 @@ abstract class LargeClusterSpec
}
"detect failure and auto-down crashed nodes in second-datacenter" taggedAs LongRunningTest in {
val unreachableNodes = nodesPerDatacenter
val downedNodes = nodesPerDatacenter
val liveNodes = nodesPerDatacenter * 4
within(30.seconds + 3.seconds * liveNodes) {
@ -289,22 +288,19 @@ abstract class LargeClusterSpec
val latch = TestLatch(nodesPerDatacenter)
systems foreach { sys
Cluster(sys).subscribe(sys.actorOf(Props(new Actor {
var gotUnreachable = Set.empty[Member]
var gotDowned = Set.empty[Member]
def receive = {
case state: CurrentClusterState
gotUnreachable = state.unreachable
checkDone()
case MemberUnreachable(m) if !latch.isOpen
gotUnreachable = gotUnreachable + m
gotDowned = gotDowned ++ state.unreachable.filter(_.status == Down)
checkDone()
case MemberDowned(m) if !latch.isOpen
gotUnreachable = gotUnreachable + m
gotDowned = gotDowned + m
checkDone()
case _ // not interesting
}
def checkDone(): Unit = if (gotUnreachable.size == unreachableNodes) {
log.info("Detected [{}] unreachable nodes in [{}], it took [{}], received [{}] gossip messages",
unreachableNodes, Cluster(sys).selfAddress, tookMillis, gossipCount(Cluster(sys)))
def checkDone(): Unit = if (gotDowned.size == downedNodes) {
log.info("Detected [{}] downed nodes in [{}], it took [{}], received [{}] gossip messages",
downedNodes, Cluster(sys).selfAddress, tookMillis, gossipCount(Cluster(sys)))
latch.countDown()
}
})), classOf[ClusterDomainEvent])
@ -318,7 +314,6 @@ abstract class LargeClusterSpec
runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) {
Await.ready(latch, remaining)
awaitCond(systems.forall(Cluster(_).readView.convergence))
val mergeCount = systems.map(sys Cluster(sys).readView.latestStats.mergeCount).sum
val counts = systems.map(sys gossipCount(Cluster(sys)))
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / nodesPerDatacenter, counts.min, counts.max)

View file

@ -57,7 +57,7 @@ abstract class LeaderLeavingSpec
enterBarrier("leader-left")
// verify that the LEADER is shut down
awaitCond(!cluster.isRunning)
awaitCond(cluster.isTerminated)
// verify that the LEADER is REMOVED
awaitCond(clusterView.status == Removed)

View file

@ -0,0 +1,149 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import language.postfixOps
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
import java.lang.management.ManagementFactory
import javax.management.InstanceNotFoundException
import javax.management.ObjectName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import scala.util.Try
object MBeanMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
akka.cluster.jmx.enabled = on
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class MBeanMultiJvmNode1 extends MBeanSpec
class MBeanMultiJvmNode2 extends MBeanSpec
class MBeanMultiJvmNode3 extends MBeanSpec
class MBeanMultiJvmNode4 extends MBeanSpec
abstract class MBeanSpec
extends MultiNodeSpec(MBeanMultiJvmSpec)
with MultiNodeClusterSpec {
import MBeanMultiJvmSpec._
import ClusterEvent._
val mbeanName = new ObjectName("akka:type=Cluster")
lazy val mbeanServer = ManagementFactory.getPlatformMBeanServer
"Cluster MBean" must {
"expose attributes" taggedAs LongRunningTest in {
val info = mbeanServer.getMBeanInfo(mbeanName)
info.getAttributes.map(_.getName).toSet must be(Set(
"ClusterStatus", "Members", "Unreachable", "MemberStatus", "Leader", "Singleton", "Available"))
enterBarrier("after-1")
}
"expose operations" taggedAs LongRunningTest in {
val info = mbeanServer.getMBeanInfo(mbeanName)
info.getOperations.map(_.getName).toSet must be(Set(
"join", "leave", "down"))
enterBarrier("after-2")
}
"change attributes after startup" taggedAs LongRunningTest in {
runOn(first) {
mbeanServer.getAttribute(mbeanName, "Available").asInstanceOf[Boolean] must be(false)
mbeanServer.getAttribute(mbeanName, "Singleton").asInstanceOf[Boolean] must be(false)
mbeanServer.getAttribute(mbeanName, "Leader") must be("")
mbeanServer.getAttribute(mbeanName, "Members") must be("")
mbeanServer.getAttribute(mbeanName, "Unreachable") must be("")
mbeanServer.getAttribute(mbeanName, "MemberStatus") must be("Removed")
}
awaitClusterUp(first)
runOn(first) {
awaitCond(mbeanServer.getAttribute(mbeanName, "MemberStatus") == "Up")
awaitCond(mbeanServer.getAttribute(mbeanName, "Leader") == address(first).toString)
mbeanServer.getAttribute(mbeanName, "Singleton").asInstanceOf[Boolean] must be(true)
mbeanServer.getAttribute(mbeanName, "Members") must be(address(first).toString)
mbeanServer.getAttribute(mbeanName, "Unreachable") must be("")
mbeanServer.getAttribute(mbeanName, "Available").asInstanceOf[Boolean] must be(true)
}
enterBarrier("after-3")
}
"support join" taggedAs LongRunningTest in {
runOn(second, third, fourth) {
mbeanServer.invoke(mbeanName, "join", Array(address(first).toString), Array("java.lang.String"))
}
enterBarrier("joined")
awaitUpConvergence(4)
assertMembers(clusterView.members, roles.map(address(_)): _*)
awaitCond(mbeanServer.getAttribute(mbeanName, "MemberStatus") == "Up")
val expectedMembers = roles.sorted.map(address(_)).mkString(",")
awaitCond(mbeanServer.getAttribute(mbeanName, "Members") == expectedMembers)
val expectedLeader = address(roleOfLeader())
awaitCond(mbeanServer.getAttribute(mbeanName, "Leader") == expectedLeader.toString)
mbeanServer.getAttribute(mbeanName, "Singleton").asInstanceOf[Boolean] must be(false)
enterBarrier("after-4")
}
"support down" taggedAs LongRunningTest in {
val fourthAddress = address(fourth)
runOn(first) {
testConductor.shutdown(fourth, 0).await
}
enterBarrier("fourth-shutdown")
runOn(first, second, third) {
awaitCond(mbeanServer.getAttribute(mbeanName, "Unreachable") == fourthAddress.toString)
val expectedMembers = Seq(first, second, third).sorted.map(address(_)).mkString(",")
awaitCond(mbeanServer.getAttribute(mbeanName, "Members") == expectedMembers)
}
enterBarrier("fourth-unreachable")
runOn(second) {
mbeanServer.invoke(mbeanName, "down", Array(fourthAddress.toString), Array("java.lang.String"))
}
enterBarrier("fourth-down")
runOn(first, second, third) {
awaitUpConvergence(3, canNotBePartOfMemberRing = List(fourthAddress))
assertMembers(clusterView.members, first, second, third)
}
enterBarrier("after-5")
}
"support leave" taggedAs LongRunningTest in within(20 seconds) {
runOn(second) {
mbeanServer.invoke(mbeanName, "leave", Array(address(third).toString), Array("java.lang.String"))
}
enterBarrier("third-left")
runOn(first, second) {
awaitUpConvergence(2)
assertMembers(clusterView.members, first, second)
val expectedMembers = Seq(first, second).sorted.map(address(_)).mkString(",")
awaitCond(mbeanServer.getAttribute(mbeanName, "Members") == expectedMembers)
}
runOn(third) {
awaitCond(cluster.isTerminated)
// mbean should be unregistered, i.e. throw InstanceNotFoundException
awaitCond(Try { mbeanServer.getMBeanInfo(mbeanName); false } recover {
case e: InstanceNotFoundException true
case _ false
} get)
}
enterBarrier("after-6")
}
}
}

View file

@ -0,0 +1,81 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfter
import scala.collection.immutable.SortedSet
import scala.concurrent.duration._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import java.util.concurrent.atomic.AtomicReference
import akka.actor.Props
import akka.actor.Actor
import akka.cluster.MemberStatus._
object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
# turn off unreachable reaper
akka.cluster.min-nr-of-members = 3""")).
withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
}
class MinMembersBeforeUpMultiJvmNode1 extends MinMembersBeforeUpSpec
class MinMembersBeforeUpMultiJvmNode2 extends MinMembersBeforeUpSpec
class MinMembersBeforeUpMultiJvmNode3 extends MinMembersBeforeUpSpec
abstract class MinMembersBeforeUpSpec
extends MultiNodeSpec(MinMembersBeforeUpMultiJvmSpec)
with MultiNodeClusterSpec {
import MinMembersBeforeUpMultiJvmSpec._
import ClusterEvent._
"Cluster leader" must {
"wait with moving members to UP until minimum number of members have joined" taggedAs LongRunningTest in {
val onUpLatch = TestLatch(1)
cluster.registerOnMemberUp(onUpLatch.countDown())
runOn(first) {
startClusterNode()
awaitCond(clusterView.status == Joining)
}
enterBarrier("first-started")
onUpLatch.isOpen must be(false)
runOn(second) {
cluster.join(first)
}
runOn(first, second) {
val expectedAddresses = Set(first, second) map address
awaitCond(clusterView.members.map(_.address) == expectedAddresses)
clusterView.members.map(_.status) must be(Set(Joining))
// and it should not change
1 to 5 foreach { _
Thread.sleep(1000)
clusterView.members.map(_.address) must be(expectedAddresses)
clusterView.members.map(_.status) must be(Set(Joining))
}
}
enterBarrier("second-joined")
runOn(third) {
cluster.join(first)
}
awaitClusterUp(first, second, third)
onUpLatch.await
enterBarrier("after-1")
}
}
}

View file

@ -64,7 +64,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
def muteLog(sys: ActorSystem = system): Unit = {
if (!sys.log.isDebugEnabled) {
Seq(".*Metrics collection has started successfully.*",
".*Hyperic SIGAR was not found on the classpath.*",
".*Metrics will be retreived from MBeans.*",
".*Cluster Node.* - registered cluster JMX MBean.*",
".*Cluster Node.* - is starting up.*",
".*Shutting down cluster Node.*",
@ -223,7 +223,9 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
within(timeout) {
awaitCond(clusterView.members.size == numberOfMembers)
awaitCond(clusterView.members.forall(_.status == MemberStatus.Up))
awaitCond(clusterView.convergence)
// clusterView.leader is updated by LeaderChanged, await that to be updated also
val expectedLeader = clusterView.members.headOption.map(_.address)
awaitCond(clusterView.leader == expectedLeader)
if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set
awaitCond(
canNotBePartOfMemberRing forall (address !(clusterView.members exists (_.address == address))))

View file

@ -51,7 +51,7 @@ abstract class NodeLeavingAndExitingAndBeingRemovedSpec
runOn(second) {
// verify that the second node is shut down and has status REMOVED
awaitCond(!cluster.isRunning, reaperWaitingTime)
awaitCond(cluster.isTerminated, reaperWaitingTime)
awaitCond(clusterView.status == MemberStatus.Removed, reaperWaitingTime)
}

View file

@ -43,7 +43,6 @@ abstract class NodeMembershipSpec
awaitCond {
clusterView.members.forall(_.status == MemberStatus.Up)
}
awaitCond(clusterView.convergence)
}
enterBarrier("after-1")
@ -60,7 +59,6 @@ abstract class NodeMembershipSpec
awaitCond {
clusterView.members.forall(_.status == MemberStatus.Up)
}
awaitCond(clusterView.convergence)
enterBarrier("after-2")
}

View file

@ -59,7 +59,7 @@ abstract class SplitBrainSpec(multiNodeConfig: SplitBrainMultiNodeConfig)
val side1 = Vector(first, second)
val side2 = Vector(third, fourth, fifth)
"A cluster of 5 members" must {
"A cluster of 5 members" ignore {
"reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(first, second, third, fourth, fifth)

View file

@ -114,7 +114,6 @@ abstract class TransitionSpec
startClusterNode()
awaitCond(clusterView.isSingletonCluster)
awaitMemberStatus(myself, Joining)
awaitCond(clusterView.convergence)
leaderActions()
awaitMemberStatus(myself, Up)
}
@ -133,14 +132,13 @@ abstract class TransitionSpec
awaitMemberStatus(first, Up)
awaitMemberStatus(second, Joining)
awaitCond(seenLatestGossip == Set(first, second))
clusterView.convergence must be(true)
}
enterBarrier("convergence-joining-2")
runOn(leader(first, second)) {
leaderActions()
awaitMemberStatus(first, Up)
awaitMemberStatus(second, Up)
awaitMemberStatus(second, Joining)
}
enterBarrier("leader-actions-2")
@ -150,7 +148,6 @@ abstract class TransitionSpec
awaitMemberStatus(second, Up)
awaitCond(seenLatestGossip == Set(first, second))
awaitMemberStatus(first, Up)
clusterView.convergence must be(true)
}
enterBarrier("after-2")
@ -163,10 +160,7 @@ abstract class TransitionSpec
}
runOn(second, third) {
// gossip chat from the join will synchronize the views
awaitMembers(first, second, third)
awaitMemberStatus(third, Joining)
awaitCond(seenLatestGossip == Set(second, third))
clusterView.convergence must be(false)
}
enterBarrier("third-joined-second")
@ -177,7 +171,6 @@ abstract class TransitionSpec
awaitMemberStatus(third, Joining)
awaitMemberStatus(second, Up)
awaitCond(seenLatestGossip == Set(first, second, third))
clusterView.convergence must be(true)
}
first gossipTo third
@ -187,7 +180,6 @@ abstract class TransitionSpec
awaitMemberStatus(second, Up)
awaitMemberStatus(third, Joining)
awaitCond(seenLatestGossip == Set(first, second, third))
clusterView.convergence must be(true)
}
enterBarrier("convergence-joining-3")
@ -196,16 +188,15 @@ abstract class TransitionSpec
leaderActions()
awaitMemberStatus(first, Up)
awaitMemberStatus(second, Up)
awaitMemberStatus(third, Up)
awaitMemberStatus(third, Joining)
}
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) {
awaitMemberStatus(third, Up)
awaitMemberStatus(third, Joining)
awaitCond(seenLatestGossip == Set(leader(first, second, third), myself))
clusterView.convergence must be(false)
}
// first non-leader gossipTo the other non-leader
@ -217,7 +208,6 @@ abstract class TransitionSpec
runOn(nonLeader(first, second, third).tail.head) {
awaitMemberStatus(third, Up)
awaitCond(seenLatestGossip == Set(first, second, third))
clusterView.convergence must be(true)
}
// first non-leader gossipTo the leader
@ -227,7 +217,6 @@ abstract class TransitionSpec
awaitMemberStatus(second, Up)
awaitMemberStatus(third, Up)
awaitCond(seenLatestGossip == Set(first, second, third))
clusterView.convergence must be(true)
}
enterBarrier("after-3")
@ -247,12 +236,10 @@ abstract class TransitionSpec
runOn(first, third) {
awaitCond(clusterView.unreachableMembers.contains(Member(second, Up)))
awaitCond(!clusterView.convergence)
}
runOn(first) {
cluster.down(second)
awaitMemberStatus(second, Down)
}
enterBarrier("after-second-down")
@ -263,7 +250,6 @@ abstract class TransitionSpec
awaitCond(clusterView.unreachableMembers.contains(Member(second, Down)))
awaitMemberStatus(second, Down)
awaitCond(seenLatestGossip == Set(first, third))
clusterView.convergence must be(true)
}
enterBarrier("after-6")

View file

@ -67,7 +67,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
enterBarrier("after_" + endBarrierNumber)
}
"A cluster of " + roles.size + " members" must {
"A cluster of " + roles.size + " members" ignore {
"reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(roles: _*)
@ -101,7 +101,6 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
members.forall(_.status == MemberStatus.Up)
})
clusterView.unreachableMembers.map(_.address) must be((allButVictim map address).toSet)
clusterView.convergence must be(false)
}
}
@ -119,8 +118,6 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
// still one unreachable
clusterView.unreachableMembers.size must be(1)
clusterView.unreachableMembers.head.address must be(node(victim).address)
// and therefore no convergence
clusterView.convergence must be(false)
}
}

View file

@ -0,0 +1,218 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.routing
import language.postfixOps
import java.lang.management.ManagementFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.MultiNodeClusterSpec
import akka.cluster.NodeMetrics
import akka.pattern.ask
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
import akka.routing.CurrentRoutees
import akka.routing.FromConfig
import akka.routing.RouterRoutees
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
class Routee extends Actor {
def receive = {
case _ sender ! Reply(Cluster(context.system).selfAddress)
}
}
class Memory extends Actor with ActorLogging {
var usedMemory: Array[Array[Int]] = _
def receive = {
case AllocateMemory
val heap = ManagementFactory.getMemoryMXBean.getHeapMemoryUsage
// getMax can be undefined (-1)
val max = math.max(heap.getMax, heap.getCommitted)
val used = heap.getUsed
log.debug("used heap before: [{}] bytes, of max [{}]", used, heap.getMax)
// allocate 70% of free space
val allocateBytes = (0.7 * (max - used)).toInt
val numberOfArrays = allocateBytes / 1024
usedMemory = Array.ofDim(numberOfArrays, 248) // each 248 element Int array will use ~ 1 kB
log.debug("used heap after: [{}] bytes", ManagementFactory.getMemoryMXBean.getHeapMemoryUsage.getUsed)
sender ! "done"
}
}
case object AllocateMemory
case class Reply(address: Address)
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
akka.cluster.metrics.collect-interval = 1s
akka.cluster.metrics.gossip-interval = 1s
akka.cluster.metrics.moving-average-half-life = 2s
akka.actor.deployment {
/router3 = {
router = adaptive
metrics-selector = cpu
nr-of-instances = 9
}
/router4 = {
router = adaptive
metrics-selector = "akka.cluster.routing.TestCustomMetricsSelector"
nr-of-instances = 10
cluster {
enabled = on
max-nr-of-instances-per-node = 2
}
}
}
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class TestCustomMetricsSelector(config: Config) extends MetricsSelector {
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty
}
class AdaptiveLoadBalancingRouterMultiJvmNode1 extends AdaptiveLoadBalancingRouterSpec
class AdaptiveLoadBalancingRouterMultiJvmNode2 extends AdaptiveLoadBalancingRouterSpec
class AdaptiveLoadBalancingRouterMultiJvmNode3 extends AdaptiveLoadBalancingRouterSpec
abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoadBalancingRouterMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender with DefaultTimeout {
import AdaptiveLoadBalancingRouterMultiJvmSpec._
def currentRoutees(router: ActorRef) =
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
def receiveReplies(expectedReplies: Int): Map[Address, Int] = {
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
(receiveWhile(5 seconds, messages = expectedReplies) {
case Reply(address) address
}).foldLeft(zero) {
case (replyMap, address) replyMap + (address -> (replyMap(address) + 1))
}
}
/**
* Fills in self address for local ActorRef
*/
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
case Address(_, _, None, None) cluster.selfAddress
case a a
}
def startRouter(name: String): ActorRef = {
val router = system.actorOf(Props[Routee].withRouter(ClusterRouterConfig(
local = AdaptiveLoadBalancingRouter(HeapMetricsSelector),
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1))), name)
awaitCond {
// it may take some time until router receives cluster member events
currentRoutees(router).size == roles.size
}
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
router
}
"A cluster with a AdaptiveLoadBalancingRouter" must {
"start cluster nodes" taggedAs LongRunningTest in {
awaitClusterUp(roles: _*)
enterBarrier("after-1")
}
"use all nodes in the cluster when not overloaded" taggedAs LongRunningTest in {
runOn(first) {
val router1 = startRouter("router1")
// collect some metrics before we start
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
val iterationCount = 100
1 to iterationCount foreach { _
router1 ! "hit"
// wait a while between each message, since metrics is collected periodically
Thread.sleep(10)
}
val replies = receiveReplies(iterationCount)
replies(first) must be > (0)
replies(second) must be > (0)
replies(third) must be > (0)
replies.values.sum must be(iterationCount)
}
enterBarrier("after-2")
}
"prefer node with more free heap capacity" taggedAs LongRunningTest in {
System.gc()
enterBarrier("gc")
runOn(second) {
within(20.seconds) {
system.actorOf(Props[Memory], "memory") ! AllocateMemory
expectMsg("done")
}
}
enterBarrier("heap-allocated")
runOn(first) {
val router2 = startRouter("router2")
router2
// collect some metrics before we start
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
val iterationCount = 3000
1 to iterationCount foreach { _
router2 ! "hit"
}
val replies = receiveReplies(iterationCount)
replies(third) must be > (replies(second))
replies.values.sum must be(iterationCount)
}
enterBarrier("after-3")
}
"create routees from configuration" taggedAs LongRunningTest in {
runOn(first) {
val router3 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router3")
awaitCond {
// it may take some time until router receives cluster member events
currentRoutees(router3).size == 9
}
currentRoutees(router3).map(fullAddress).toSet must be(Set(address(first)))
}
enterBarrier("after-4")
}
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
runOn(first) {
val router4 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router4")
awaitCond {
// it may take some time until router receives cluster member events
currentRoutees(router4).size == 6
}
currentRoutees(router4).map(fullAddress).toSet must be(Set(
address(first), address(second), address(third)))
}
enterBarrier("after-5")
}
}
}

View file

@ -36,6 +36,7 @@ class ClusterConfigSpec extends AkkaSpec {
JoinTimeout must be(60 seconds)
AutoJoin must be(true)
AutoDown must be(false)
MinNrOfMembers must be(1)
JmxEnabled must be(true)
UseDispatcher must be(Dispatchers.DefaultDispatcherId)
GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001)
@ -47,9 +48,10 @@ class ClusterConfigSpec extends AkkaSpec {
callTimeout = 2 seconds,
resetTimeout = 30 seconds))
MetricsEnabled must be(true)
MetricsCollectorClass must be(classOf[SigarMetricsCollector].getName)
MetricsInterval must be(3 seconds)
MetricsGossipInterval must be(3 seconds)
MetricsRateOfDecay must be(10)
MetricsMovingAverageHalfLife must be(12 seconds)
}
}
}

View file

@ -18,21 +18,9 @@ import akka.testkit.ImplicitSender
import akka.actor.ActorRef
import akka.testkit.TestProbe
object ClusterDomainEventPublisherSpec {
val config = """
akka.cluster.auto-join = off
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off
akka.remote.netty.port = 0
"""
case class GossipTo(address: Address)
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config)
class ClusterDomainEventPublisherSpec extends AkkaSpec
with BeforeAndAfterEach with ImplicitSender {
import ClusterDomainEventPublisherSpec._
var publisher: ActorRef = _
val a1 = Member(Address("akka", "sys", "a", 2552), Up)
@ -54,6 +42,10 @@ class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublish
override def beforeEach(): Unit = {
publisher = system.actorOf(Props[ClusterDomainEventPublisher])
publisher ! PublishChanges(g0)
expectMsg(MemberUp(a1))
expectMsg(LeaderChanged(Some(a1.address)))
expectMsgType[SeenChanged]
}
override def afterEach(): Unit = {
@ -62,59 +54,63 @@ class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublish
"ClusterDomainEventPublisher" must {
"publish MemberUp when member status changed to Up" in {
publisher ! PublishChanges(g1, g2)
expectMsg(MemberUp(c2))
expectMsg(ConvergenceChanged(false))
"not publish MemberUp when there is no convergence" in {
publisher ! PublishChanges(g2)
expectMsgType[SeenChanged]
}
"publish convergence true when all seen it" in {
publisher ! PublishChanges(g2, g3)
expectMsg(ConvergenceChanged(true))
"publish MemberEvents when there is convergence" in {
publisher ! PublishChanges(g2)
expectMsgType[SeenChanged]
publisher ! PublishChanges(g3)
expectMsg(MemberUp(b1))
expectMsg(MemberUp(c2))
expectMsgType[SeenChanged]
}
"publish leader changed when new leader after convergence" in {
publisher ! PublishChanges(g3, g4)
expectMsg(MemberUp(d1))
expectMsg(ConvergenceChanged(false))
publisher ! PublishChanges(g4)
expectMsgType[SeenChanged]
expectNoMsg(1 second)
publisher ! PublishChanges(g4, g5)
publisher ! PublishChanges(g5)
expectMsg(MemberUp(d1))
expectMsg(MemberUp(b1))
expectMsg(MemberUp(c2))
expectMsg(LeaderChanged(Some(d1.address)))
expectMsg(ConvergenceChanged(true))
expectMsgType[SeenChanged]
}
"publish leader changed when new leader and convergence both before and after" in {
// convergence both before and after
publisher ! PublishChanges(g3, g5)
publisher ! PublishChanges(g3)
expectMsg(MemberUp(b1))
expectMsg(MemberUp(c2))
expectMsgType[SeenChanged]
publisher ! PublishChanges(g5)
expectMsg(MemberUp(d1))
expectMsg(LeaderChanged(Some(d1.address)))
expectMsgType[SeenChanged]
}
"not publish leader changed when not convergence" in {
publisher ! PublishChanges(g2, g4)
expectMsg(MemberUp(d1))
publisher ! PublishChanges(g4)
expectMsgType[SeenChanged]
expectNoMsg(1 second)
}
"not publish leader changed when changed convergence but still same leader" in {
publisher ! PublishChanges(g2, g5)
publisher ! PublishChanges(g5)
expectMsg(MemberUp(d1))
expectMsg(MemberUp(b1))
expectMsg(MemberUp(c2))
expectMsg(LeaderChanged(Some(d1.address)))
expectMsg(ConvergenceChanged(true))
expectMsgType[SeenChanged]
publisher ! PublishChanges(g5, g4)
expectMsg(ConvergenceChanged(false))
publisher ! PublishChanges(g4)
expectMsgType[SeenChanged]
publisher ! PublishChanges(g4, g5)
expectMsg(ConvergenceChanged(true))
publisher ! PublishChanges(g5)
expectMsgType[SeenChanged]
}
@ -131,12 +127,12 @@ class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublish
publisher ! Subscribe(subscriber.ref, classOf[ClusterDomainEvent])
subscriber.expectMsgType[CurrentClusterState]
publisher ! Unsubscribe(subscriber.ref, Some(classOf[ClusterDomainEvent]))
publisher ! PublishChanges(Gossip(members = SortedSet(a1)), Gossip(members = SortedSet(a1, b1)))
publisher ! PublishChanges(g3)
subscriber.expectNoMsg(1 second)
// but testActor is still subscriber
expectMsg(MemberUp(b1))
expectMsg(MemberUp(c2))
expectMsgType[SeenChanged]
}
}
}

View file

@ -17,6 +17,7 @@ class ClusterDomainEventSpec extends WordSpec with MustMatchers {
val a1 = Member(Address("akka", "sys", "a", 2552), Up)
val a2 = Member(Address("akka", "sys", "a", 2552), Joining)
val a3 = Member(Address("akka", "sys", "a", 2552), Removed)
val b1 = Member(Address("akka", "sys", "b", 2552), Up)
val b2 = Member(Address("akka", "sys", "b", 2552), Removed)
val b3 = Member(Address("akka", "sys", "b", 2552), Down)
@ -28,61 +29,82 @@ class ClusterDomainEventSpec extends WordSpec with MustMatchers {
val e2 = Member(Address("akka", "sys", "e", 2552), Up)
val e3 = Member(Address("akka", "sys", "e", 2552), Down)
def converge(gossip: Gossip): (Gossip, Set[Address]) =
((gossip, Set.empty[Address]) /: gossip.members) { (gs, m) (gs._1.seen(m.address), gs._2 + m.address) }
"Domain events" must {
"be produced for new members" in {
"be empty for the same gossip" in {
val g1 = Gossip(members = SortedSet(a1))
val g2 = Gossip(members = SortedSet(a1, b1, e1))
diff(g1, g2) must be(Seq(MemberUp(b1), MemberJoined(e1)))
diffUnreachable(g1, g1) must be(Seq.empty)
}
"be produced for new members" in {
val (g1, _) = converge(Gossip(members = SortedSet(a1)))
val (g2, s2) = converge(Gossip(members = SortedSet(a1, b1, e1)))
diffMemberEvents(g1, g2) must be(Seq(MemberUp(b1), MemberJoined(e1)))
diffUnreachable(g1, g2) must be(Seq.empty)
diffSeen(g1, g2) must be(Seq(SeenChanged(convergence = true, seenBy = s2)))
}
"be produced for changed status of members" in {
val g1 = Gossip(members = SortedSet(a2, b1, c2))
val g2 = Gossip(members = SortedSet(a1, b1, c1, e1))
val (g1, _) = converge(Gossip(members = SortedSet(a2, b1, c2)))
val (g2, s2) = converge(Gossip(members = SortedSet(a1, b1, c1, e1)))
diff(g1, g2) must be(Seq(MemberUp(a1), MemberLeft(c1), MemberJoined(e1)))
diffMemberEvents(g1, g2) must be(Seq(MemberUp(a1), MemberLeft(c1), MemberJoined(e1)))
diffUnreachable(g1, g2) must be(Seq.empty)
diffSeen(g1, g2) must be(Seq(SeenChanged(convergence = true, seenBy = s2)))
}
"be produced for unreachable members" in {
val g1 = Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(c2)))
val g2 = Gossip(members = SortedSet(a1), overview = GossipOverview(unreachable = Set(b1, c2)))
diff(g1, g2) must be(Seq(MemberUnreachable(b1)))
}
"be produced for downed members" in {
"be produced for members in unreachable" in {
val g1 = Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(c2, e2)))
val g2 = Gossip(members = SortedSet(a1), overview = GossipOverview(unreachable = Set(c2, b3, e3)))
diff(g1, g2) must be(Seq(MemberDowned(b3), MemberDowned(e3)))
diffMemberEvents(g1, g2) must be(Seq(MemberDowned(b3), MemberDowned(e3)))
diffUnreachable(g1, g2) must be(Seq(UnreachableMember(b3)))
diffSeen(g1, g2) must be(Seq.empty)
}
"be produced for downed members" in {
val (g1, _) = converge(Gossip(members = SortedSet(a1, b1)))
val (g2, _) = converge(Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(e3))))
diffMemberEvents(g1, g2) must be(Seq(MemberDowned(e3)))
diffUnreachable(g1, g2) must be(Seq(UnreachableMember(e3)))
diffSeen(g1, g2) must be(Seq.empty)
}
"be produced for removed members" in {
val g1 = Gossip(members = SortedSet(a1, d1), overview = GossipOverview(unreachable = Set(c2)))
val g2 = Gossip(members = SortedSet(a1), overview = GossipOverview(unreachable = Set(c2)))
val (g1, _) = converge(Gossip(members = SortedSet(a1, d1)))
val (g2, s2) = converge(Gossip(members = SortedSet(a1)))
diff(g1, g2) must be(Seq(MemberRemoved(d2)))
diffMemberEvents(g1, g2) must be(Seq(MemberRemoved(d2)))
diffUnreachable(g1, g2) must be(Seq.empty)
diffSeen(g1, g2) must be(Seq(SeenChanged(convergence = true, seenBy = s2)))
}
"be produced for convergence changes" in {
val g1 = Gossip(members = SortedSet(a1, b1, e1)).seen(a1.address).seen(b1.address).seen(e1.address)
val g2 = Gossip(members = SortedSet(a1, b1, e1)).seen(a1.address).seen(b1.address)
diff(g1, g2) must be(Seq(ConvergenceChanged(false),
SeenChanged(convergence = false, seenBy = Set(a1.address, b1.address))))
diff(g2, g1) must be(Seq(ConvergenceChanged(true),
SeenChanged(convergence = true, seenBy = Set(a1.address, b1.address, e1.address))))
diffMemberEvents(g1, g2) must be(Seq.empty)
diffUnreachable(g1, g2) must be(Seq.empty)
diffSeen(g1, g2) must be(Seq(SeenChanged(convergence = false, seenBy = Set(a1.address, b1.address))))
diffMemberEvents(g2, g1) must be(Seq.empty)
diffUnreachable(g2, g1) must be(Seq.empty)
diffSeen(g2, g1) must be(Seq(SeenChanged(convergence = true, seenBy = Set(a1.address, b1.address, e1.address))))
}
"be produced for leader changes" in {
val g1 = Gossip(members = SortedSet(a1, b1, e1))
val g2 = Gossip(members = SortedSet(b1, e1), overview = GossipOverview(unreachable = Set(a1)))
val g3 = g2.copy(overview = GossipOverview()).seen(b1.address).seen(e1.address)
val (g1, _) = converge(Gossip(members = SortedSet(a1, b1, e1)))
val (g2, s2) = converge(Gossip(members = SortedSet(b1, e1)))
diff(g1, g2) must be(Seq(MemberUnreachable(a1), LeaderChanged(Some(b1.address))))
diff(g2, g3) must be(Seq(ConvergenceChanged(true),
SeenChanged(convergence = true, seenBy = Set(b1.address, e1.address))))
diffMemberEvents(g1, g2) must be(Seq(MemberRemoved(a3)))
diffUnreachable(g1, g2) must be(Seq.empty)
diffSeen(g1, g2) must be(Seq(SeenChanged(convergence = true, seenBy = s2)))
diffLeader(g1, g2) must be(Seq(LeaderChanged(Some(b1.address))))
}
}
}

View file

@ -68,7 +68,6 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
clusterView.self.address must be(selfAddress)
clusterView.members.map(_.address) must be(Set(selfAddress))
clusterView.status must be(MemberStatus.Joining)
clusterView.convergence must be(true)
leaderActions()
awaitCond(clusterView.status == MemberStatus.Up)
}

View file

@ -1,62 +0,0 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import language.postfixOps
import scala.concurrent.duration._
import akka.testkit.{ LongRunningTest, AkkaSpec }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class DataStreamSpec extends AkkaSpec(MetricsEnabledSpec.config) with AbstractClusterMetricsSpec with MetricNumericConverter {
import system.dispatcher
val collector = createMetricsCollector
val DefaultRateOfDecay = 10
"DataStream" must {
"calculate the ewma for multiple, variable, data streams" taggedAs LongRunningTest in {
val firstDataSet = collector.sample.metrics.collect { case m if m.trendable && m.isDefined m.initialize(DefaultRateOfDecay) }
var streamingDataSet = firstDataSet
val cancellable = system.scheduler.schedule(0 seconds, 100 millis) {
streamingDataSet = collector.sample.metrics.flatMap(latest streamingDataSet.collect {
case streaming if (latest.trendable && latest.isDefined) && (latest same streaming)
&& (latest.value.get != streaming.value.get) {
val updatedDataStream = streaming.average.get :+ latest.value.get
updatedDataStream.timestamp must be > (streaming.average.get.timestamp)
updatedDataStream.duration.length must be > (streaming.average.get.duration.length)
updatedDataStream.ewma must not be (streaming.average.get.ewma)
updatedDataStream.ewma must not be (latest.value.get)
streaming.copy(value = latest.value, average = Some(updatedDataStream))
}
})
}
awaitCond(firstDataSet.size == streamingDataSet.size, longDuration)
cancellable.cancel()
val finalDataSet = streamingDataSet.map(m m.name -> m).toMap
firstDataSet map {
first
val newMetric = finalDataSet(first.name)
val e1 = first.average.get
val e2 = newMetric.average.get
if (first.value.get != newMetric.value.get) {
e2.ewma must not be (first.value.get)
e2.ewma must not be (newMetric.value.get)
}
if (first.value.get.longValue > newMetric.value.get.longValue) e1.ewma.longValue must be > e2.ewma.longValue
else if (first.value.get.longValue < newMetric.value.get.longValue) e1.ewma.longValue must be < e2.ewma.longValue
}
}
"data streaming is disabled if the decay is set to 0" in {
val data = collector.sample.metrics map (_.initialize(0))
data foreach (_.average.isEmpty must be(true))
}
}
}

View file

@ -0,0 +1,101 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import language.postfixOps
import scala.concurrent.duration._
import akka.testkit.{ LongRunningTest, AkkaSpec }
import scala.concurrent.forkjoin.ThreadLocalRandom
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class EWMASpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricsCollectorFactory {
import system.dispatcher
val collector = createMetricsCollector
"DataStream" must {
"calcualate same ewma for constant values" in {
val ds = EWMA(value = 100.0, alpha = 0.18) :+
100.0 :+ 100.0 :+ 100.0
ds.value must be(100.0 plusOrMinus 0.001)
}
"calcualate correct ewma for normal decay" in {
val d0 = EWMA(value = 1000.0, alpha = 2.0 / (1 + 10))
d0.value must be(1000.0 plusOrMinus 0.01)
val d1 = d0 :+ 10.0
d1.value must be(820.0 plusOrMinus 0.01)
val d2 = d1 :+ 10.0
d2.value must be(672.73 plusOrMinus 0.01)
val d3 = d2 :+ 10.0
d3.value must be(552.23 plusOrMinus 0.01)
val d4 = d3 :+ 10.0
d4.value must be(453.64 plusOrMinus 0.01)
val dn = (1 to 100).foldLeft(d0)((d, _) d :+ 10.0)
dn.value must be(10.0 plusOrMinus 0.1)
}
"calculate ewma for alpha 1.0, max bias towards latest value" in {
val d0 = EWMA(value = 100.0, alpha = 1.0)
d0.value must be(100.0 plusOrMinus 0.01)
val d1 = d0 :+ 1.0
d1.value must be(1.0 plusOrMinus 0.01)
val d2 = d1 :+ 57.0
d2.value must be(57.0 plusOrMinus 0.01)
val d3 = d2 :+ 10.0
d3.value must be(10.0 plusOrMinus 0.01)
}
"calculate alpha from half-life and collect interval" in {
// according to http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average
val expectedAlpha = 0.1
// alpha = 2.0 / (1 + N)
val n = 19
val halfLife = n.toDouble / 2.8854
val collectInterval = 1.second
val halfLifeDuration = (halfLife * 1000).millis
EWMA.alpha(halfLifeDuration, collectInterval) must be(expectedAlpha plusOrMinus 0.001)
}
"calculate sane alpha from short half-life" in {
val alpha = EWMA.alpha(1.millis, 3.seconds)
alpha must be <= (1.0)
alpha must be >= (0.0)
alpha must be(1.0 plusOrMinus 0.001)
}
"calculate sane alpha from long half-life" in {
val alpha = EWMA.alpha(1.day, 3.seconds)
alpha must be <= (1.0)
alpha must be >= (0.0)
alpha must be(0.0 plusOrMinus 0.001)
}
"calculate the ewma for multiple, variable, data streams" taggedAs LongRunningTest in {
var streamingDataSet = Map.empty[String, Metric]
var usedMemory = Array.empty[Byte]
(1 to 50) foreach { _
// wait a while between each message to give the metrics a chance to change
Thread.sleep(100)
usedMemory = usedMemory ++ Array.fill(1024)(ThreadLocalRandom.current.nextInt(127).toByte)
val changes = collector.sample.metrics.flatMap { latest
streamingDataSet.get(latest.name) match {
case None Some(latest)
case Some(previous)
if (latest.isSmooth && latest.value != previous.value) {
val updated = previous :+ latest
updated.isSmooth must be(true)
updated.smoothValue must not be (previous.smoothValue)
Some(updated)
} else None
}
}
streamingDataSet ++= changes.map(m m.name -> m)
}
}
}
}

View file

@ -28,8 +28,11 @@ class GossipSpec extends WordSpec with MustMatchers {
"A Gossip" must {
"merge members by status priority" in {
"reach convergence when it's empty" in {
Gossip().convergence must be(true)
}
"merge members by status priority" in {
val g1 = Gossip(members = SortedSet(a1, c1, e1))
val g2 = Gossip(members = SortedSet(a2, c2, e2))
@ -44,7 +47,6 @@ class GossipSpec extends WordSpec with MustMatchers {
}
"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)))

View file

@ -4,40 +4,35 @@
package akka.cluster
import akka.testkit.{ ImplicitSender, AkkaSpec }
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.cluster.StandardMetrics._
import scala.util.Failure
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricNumericConverterSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricNumericConverter with ImplicitSender with AbstractClusterMetricsSpec {
class MetricNumericConverterSpec extends WordSpec with MustMatchers with MetricNumericConverter {
"MetricNumericConverter" must {
val collector = createMetricsCollector
"convert" in {
convert(0).isLeft must be(true)
convert(1).left.get must be(1)
convert(1L).isLeft must be(true)
convert(0.0).isRight must be(true)
convertNumber(0).isLeft must be(true)
convertNumber(1).left.get must be(1)
convertNumber(1L).isLeft must be(true)
convertNumber(0.0).isRight must be(true)
}
"define a new metric" in {
val metric = Metric("heap-memory-used", Some(0L))
metric.initializable must be(true)
metric.name must not be (null)
metric.average.isEmpty must be(true)
metric.trendable must be(true)
if (collector.isSigar) {
val cores = collector.totalCores
cores.isDefined must be(true)
cores.value.get.intValue must be > (0)
cores.initializable must be(false)
}
val Some(metric) = Metric.create(HeapMemoryUsed, 256L, decayFactor = Some(0.18))
metric.name must be(HeapMemoryUsed)
metric.value must be(256L)
metric.isSmooth must be(true)
metric.smoothValue must be(256.0 plusOrMinus 0.0001)
}
"define an undefined value with a None " in {
Metric("x", Some(-1)).value.isDefined must be(false)
Metric("x", Some(java.lang.Double.NaN)).value.isDefined must be(false)
Metric("x", None).isDefined must be(false)
Metric.create("x", -1, None).isDefined must be(false)
Metric.create("x", java.lang.Double.NaN, None).isDefined must be(false)
Metric.create("x", Failure(new RuntimeException), None).isDefined must be(false)
}
"recognize whether a metric value is defined" in {
@ -47,6 +42,7 @@ class MetricNumericConverterSpec extends AkkaSpec(MetricsEnabledSpec.config) wit
"recognize whether a metric value is not defined" in {
defined(-1) must be(false)
defined(-1.0) must be(false)
defined(Double.NaN) must be(false)
}
}

View file

@ -0,0 +1,69 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import scala.util.Try
import akka.actor.Address
import akka.testkit.AkkaSpec
import akka.cluster.StandardMetrics._
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricValuesSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricsCollectorFactory {
val collector = createMetricsCollector
val node1 = NodeMetrics(Address("akka", "sys", "a", 2554), 1, collector.sample.metrics)
val node2 = NodeMetrics(Address("akka", "sys", "a", 2555), 1, collector.sample.metrics)
val nodes: Seq[NodeMetrics] = {
(1 to 100).foldLeft(List(node1, node2)) { (nodes, _)
nodes map { n
n.copy(metrics = collector.sample.metrics.flatMap(latest n.metrics.collect {
case streaming if latest sameAs streaming streaming :+ latest
}))
}
}
}
"NodeMetrics.MetricValues" must {
"extract expected metrics for load balancing" in {
val stream1 = node2.metric(HeapMemoryCommitted).get.value.longValue
val stream2 = node1.metric(HeapMemoryUsed).get.value.longValue
stream1 must be >= (stream2)
}
"extract expected MetricValue types for load balancing" in {
nodes foreach { node
node match {
case HeapMemory(address, _, used, committed, Some(max))
committed must be >= (used)
used must be <= (max)
committed must be <= (max)
// extract is the java api
StandardMetrics.extractHeapMemory(node) must not be (null)
case HeapMemory(address, _, used, committed, None)
used must be > (0L)
committed must be > (0L)
// extract is the java api
StandardMetrics.extractCpu(node) must not be (null)
}
node match {
case Cpu(address, _, systemLoadAverageOption, cpuCombinedOption, processors)
processors must be > (0)
if (systemLoadAverageOption.isDefined)
systemLoadAverageOption.get must be >= (0.0)
if (cpuCombinedOption.isDefined) {
cpuCombinedOption.get must be <= (1.0)
cpuCombinedOption.get must be >= (0.0)
}
// extract is the java api
StandardMetrics.extractCpu(node) must not be (null)
}
}
}
}
}

View file

@ -1,4 +1,5 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
@ -13,57 +14,48 @@ import scala.util.{ Success, Try, Failure }
import akka.actor._
import akka.testkit._
import akka.cluster.StandardMetrics._
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
object MetricsEnabledSpec {
val config = """
akka.cluster.metrics.enabled = on
akka.cluster.metrics.metrics-interval = 1 s
akka.cluster.metrics.collect-interval = 1 s
akka.cluster.metrics.gossip-interval = 1 s
akka.cluster.metrics.rate-of-decay = 10
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
"""
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with AbstractClusterMetricsSpec with MetricSpec {
class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with MetricsCollectorFactory {
import system.dispatcher
val collector = createMetricsCollector
"Metric must" must {
"create and initialize a new metric or merge an existing one" in {
for (i 0 to samples) {
val metrics = collector.sample.metrics
assertCreatedUninitialized(metrics)
assertInitialized(window, metrics map (_.initialize(window)))
}
}
"merge 2 metrics that are tracking the same metric" in {
for (i 0 to samples) {
for (i 1 to 20) {
val sample1 = collector.sample.metrics
val sample2 = collector.sample.metrics
var merged = sample2 flatMap (latest sample1 collect {
case peer if latest same peer {
val merged12 = sample2 flatMap (latest sample1 collect {
case peer if latest sameAs peer
val m = peer :+ latest
assertMerged(latest, peer, m)
m.value must be(latest.value)
m.isSmooth must be(peer.isSmooth || latest.isSmooth)
m
}
})
val sample3 = collector.sample.metrics map (_.initialize(window))
val sample4 = collector.sample.metrics map (_.initialize(window))
merged = sample4 flatMap (latest sample3 collect {
case peer if latest same peer {
val sample3 = collector.sample.metrics
val sample4 = collector.sample.metrics
val merged34 = sample4 flatMap (latest sample3 collect {
case peer if latest sameAs peer
val m = peer :+ latest
assertMerged(latest, peer, m)
m.value must be(latest.value)
m.isSmooth must be(peer.isSmooth || latest.isSmooth)
m
}
})
merged.size must be(sample3.size)
merged.size must be(sample4.size)
}
}
}
@ -76,158 +68,65 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl
"collect accurate metrics for a node" in {
val sample = collector.sample
assertExpectedSampleSize(collector.isSigar, window, sample)
val metrics = sample.metrics.collect { case m if m.isDefined (m.name, m.value.get) }
val used = metrics collectFirst { case ("heap-memory-used", b) b }
val committed = metrics collectFirst { case ("heap-memory-committed", b) b }
val metrics = sample.metrics.collect { case m (m.name, m.value) }
val used = metrics collectFirst { case (HeapMemoryUsed, b) b }
val committed = metrics collectFirst { case (HeapMemoryCommitted, b) b }
metrics foreach {
case ("total-cores", b) b.intValue must be > (0)
case ("network-max-rx", b) b.longValue must be > (0L)
case ("network-max-tx", b) b.longValue must be > (0L)
case ("system-load-average", b) b.doubleValue must be >= (0.0)
case ("processors", b) b.intValue must be >= (0)
case ("heap-memory-used", b) b.longValue must be >= (0L)
case ("heap-memory-committed", b) b.longValue must be > (0L)
case ("cpu-combined", b)
b.doubleValue must be <= (1.0)
b.doubleValue must be >= (0.0)
case ("heap-memory-max", b)
case (SystemLoadAverage, b) b.doubleValue must be >= (0.0)
case (Processors, b) b.intValue must be >= (0)
case (HeapMemoryUsed, b) b.longValue must be >= (0L)
case (HeapMemoryCommitted, b) b.longValue must be > (0L)
case (HeapMemoryMax, b)
b.longValue must be > (0L)
used.get.longValue must be <= (b.longValue)
committed.get.longValue must be <= (b.longValue)
}
}
case (CpuCombined, b)
b.doubleValue must be <= (1.0)
b.doubleValue must be >= (0.0)
"collect SIGAR metrics if it is on the classpath" in {
if (collector.isSigar) {
// combined cpu may or may not be defined on a given sampling
// systemLoadAverage is SIGAR present
collector.systemLoadAverage.isDefined must be(true)
collector.networkStats.nonEmpty must be(true)
collector.networkMaxRx.isDefined must be(true)
collector.networkMaxTx.isDefined must be(true)
collector.totalCores.isDefined must be(true)
}
}
"collect JMX metrics" in {
// heap max may be undefined depending on the OS
// systemLoadAverage is JMX if SIGAR not present, but not available on all OS
collector.used.isDefined must be(true)
collector.committed.isDefined must be(true)
collector.processors.isDefined must be(true)
// systemLoadAverage is JMX when SIGAR not present, but
// it's not present on all platforms
val c = collector.asInstanceOf[JmxMetricsCollector]
val heap = c.heapMemoryUsage
c.heapUsed(heap).isDefined must be(true)
c.heapCommitted(heap).isDefined must be(true)
c.processors.isDefined must be(true)
}
"collect [" + samples + "] node metrics samples in an acceptable duration" taggedAs LongRunningTest in {
val latch = TestLatch(samples)
val task = system.scheduler.schedule(0 seconds, interval) {
"collect 50 node metrics samples in an acceptable duration" taggedAs LongRunningTest in within(7 seconds) {
(1 to 50) foreach { _
val sample = collector.sample
assertCreatedUninitialized(sample.metrics)
assertExpectedSampleSize(collector.isSigar, window, sample)
latch.countDown()
sample.metrics.size must be >= (3)
Thread.sleep(100)
}
Await.ready(latch, longDuration)
task.cancel()
}
}
}
trait MetricSpec extends WordSpec with MustMatchers {
def assertMasterMetricsAgainstGossipMetrics(master: Set[NodeMetrics], gossip: MetricsGossip): Unit = {
val masterMetrics = collectNodeMetrics(master)
val gossipMetrics = collectNodeMetrics(gossip.nodes)
gossipMetrics.size must be(masterMetrics.size plusOrMinus 1) // combined cpu
}
def assertExpectedNodeAddresses(gossip: MetricsGossip, nodes: Set[NodeMetrics]): Unit =
gossip.nodes.map(_.address) must be(nodes.map(_.address))
def assertExpectedSampleSize(isSigar: Boolean, gossip: MetricsGossip): Unit =
gossip.nodes.foreach(n assertExpectedSampleSize(isSigar, gossip.rateOfDecay, n))
def assertCreatedUninitialized(gossip: MetricsGossip): Unit =
gossip.nodes.foreach(n assertCreatedUninitialized(n.metrics.filterNot(_.trendable)))
def assertInitialized(gossip: MetricsGossip): Unit =
gossip.nodes.foreach(n assertInitialized(gossip.rateOfDecay, n.metrics))
def assertCreatedUninitialized(metrics: Set[Metric]): Unit = {
metrics.size must be > (0)
metrics foreach { m
m.average.isEmpty must be(true)
if (m.value.isDefined) m.isDefined must be(true)
if (m.initializable) (m.trendable && m.isDefined && m.average.isEmpty) must be(true)
}
}
def assertInitialized(decay: Int, metrics: Set[Metric]): Unit = if (decay > 0) metrics.filter(_.trendable) foreach { m
m.initializable must be(false)
if (m.isDefined) m.average.isDefined must be(true)
}
def assertMerged(latest: Metric, peer: Metric, merged: Metric): Unit = if (latest same peer) {
if (latest.isDefined) {
if (peer.isDefined) {
merged.isDefined must be(true)
merged.value.get must be(latest.value.get)
if (latest.trendable) {
if (latest.initializable) merged.average.isEmpty must be(true)
else merged.average.isDefined must be(true)
}
} else {
merged.isDefined must be(true)
merged.value.get must be(latest.value.get)
if (latest.average.isDefined) merged.average.get must be(latest.average.get)
else merged.average.isEmpty must be(true)
}
} else {
if (peer.isDefined) {
merged.isDefined must be(true)
merged.value.get must be(peer.value.get)
if (peer.trendable) {
if (peer.initializable) merged.average.isEmpty must be(true)
else merged.average.isDefined must be(true)
}
} else {
merged.isDefined must be(false)
merged.average.isEmpty must be(true)
}
}
}
def assertExpectedSampleSize(isSigar: Boolean, decay: Int, node: NodeMetrics): Unit = {
node.metrics.size must be(9)
val metrics = node.metrics.filter(_.isDefined)
if (isSigar) { // combined cpu + jmx max heap
metrics.size must be >= (7)
metrics.size must be <= (9)
} else { // jmx max heap
metrics.size must be >= (4)
metrics.size must be <= (5)
}
if (decay > 0) metrics.collect { case m if m.trendable && (!m.initializable) m }.foreach(_.average.isDefined must be(true))
}
def collectNodeMetrics(nodes: Set[NodeMetrics]): immutable.Seq[Metric] =
nodes.foldLeft(Vector[Metric]()) {
case (r, n) r ++ n.metrics.filter(_.isDefined)
}
}
trait AbstractClusterMetricsSpec extends DefaultTimeout {
this: AkkaSpec
val selfAddress = new Address("akka", "localhost")
val window = 49
val interval: FiniteDuration = 100 millis
val longDuration = 120 seconds // for long running tests
val samples = 100
def createMetricsCollector: MetricsCollector = MetricsCollector(selfAddress, log, system.asInstanceOf[ExtendedActorSystem].dynamicAccess)
/**
* Used when testing metrics without full cluster
*/
trait MetricsCollectorFactory { this: AkkaSpec
private def extendedActorSystem = system.asInstanceOf[ExtendedActorSystem]
def selfAddress = extendedActorSystem.provider.rootPath.address
val defaultDecayFactor = 2.0 / (1 + 10)
def createMetricsCollector: MetricsCollector =
Try(new SigarMetricsCollector(selfAddress, defaultDecayFactor,
extendedActorSystem.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Nil))).
recover {
case e
log.debug("Metrics will be retreived from MBeans, Sigar failed to load. Reason: " + e)
new JmxMetricsCollector(selfAddress, defaultDecayFactor)
}.get
def isSigar(collector: MetricsCollector): Boolean = collector.isInstanceOf[SigarMetricsCollector]
}

View file

@ -12,95 +12,95 @@ import akka.actor.Address
import java.lang.System.{ currentTimeMillis newTimestamp }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsGossipSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with AbstractClusterMetricsSpec with MetricSpec {
class MetricsGossipSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with MetricsCollectorFactory {
val collector = createMetricsCollector
"A MetricsGossip" must {
"add and initialize new NodeMetrics" in {
"add new NodeMetrics" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
var localGossip = MetricsGossip(window)
localGossip :+= m1
localGossip.nodes.size must be(1)
localGossip.nodeKeys.size must be(localGossip.nodes.size)
assertMasterMetricsAgainstGossipMetrics(Set(m1), localGossip)
assertExpectedSampleSize(collector.isSigar, localGossip)
assertInitialized(localGossip.rateOfDecay, collectNodeMetrics(localGossip.nodes).toSet)
m1.metrics.size must be > (3)
m2.metrics.size must be > (3)
localGossip :+= m2
localGossip.nodes.size must be(2)
localGossip.nodeKeys.size must be(localGossip.nodes.size)
assertMasterMetricsAgainstGossipMetrics(Set(m1, m2), localGossip)
assertExpectedSampleSize(collector.isSigar, localGossip)
assertInitialized(localGossip.rateOfDecay, collectNodeMetrics(localGossip.nodes).toSet)
val g1 = MetricsGossip.empty :+ m1
g1.nodes.size must be(1)
g1.nodeMetricsFor(m1.address).map(_.metrics) must be(Some(m1.metrics))
val g2 = g1 :+ m2
g2.nodes.size must be(2)
g2.nodeMetricsFor(m1.address).map(_.metrics) must be(Some(m1.metrics))
g2.nodeMetricsFor(m2.address).map(_.metrics) must be(Some(m2.metrics))
}
"merge peer metrics" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
var remoteGossip = MetricsGossip(window)
remoteGossip :+= m1
remoteGossip :+= m2
remoteGossip.nodes.size must be(2)
val beforeMergeNodes = remoteGossip.nodes
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size must be(2)
val beforeMergeNodes = g1.nodes
val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = newTimestamp)
remoteGossip :+= m2Updated // merge peers
remoteGossip.nodes.size must be(2)
assertMasterMetricsAgainstGossipMetrics(beforeMergeNodes, remoteGossip)
assertExpectedSampleSize(collector.isSigar, remoteGossip)
remoteGossip.nodes collect { case peer if peer.address == m2.address peer.timestamp must be(m2Updated.timestamp) }
val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = m2.timestamp + 1000)
val g2 = g1 :+ m2Updated // merge peers
g2.nodes.size must be(2)
g2.nodeMetricsFor(m1.address).map(_.metrics) must be(Some(m1.metrics))
g2.nodeMetricsFor(m2.address).map(_.metrics) must be(Some(m2Updated.metrics))
g2.nodes collect { case peer if peer.address == m2.address peer.timestamp must be(m2Updated.timestamp) }
}
"merge an existing metric set for a node and update node ring" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
val m3 = NodeMetrics(Address("akka", "sys", "a", 2556), newTimestamp, collector.sample.metrics)
val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = newTimestamp)
val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = m2.timestamp + 1000)
var localGossip = MetricsGossip(window)
localGossip :+= m1
localGossip :+= m2
val g1 = MetricsGossip.empty :+ m1 :+ m2
val g2 = MetricsGossip.empty :+ m3 :+ m2Updated
var remoteGossip = MetricsGossip(window)
remoteGossip :+= m3
remoteGossip :+= m2Updated
localGossip.nodeKeys.contains(m1.address) must be(true)
remoteGossip.nodeKeys.contains(m3.address) must be(true)
g1.nodes.map(_.address) must be(Set(m1.address, m2.address))
// must contain nodes 1,3, and the most recent version of 2
val mergedGossip = localGossip merge remoteGossip
mergedGossip.nodes.size must be(3)
assertExpectedNodeAddresses(mergedGossip, Set(m1, m2, m3))
assertExpectedSampleSize(collector.isSigar, mergedGossip)
assertCreatedUninitialized(mergedGossip)
assertInitialized(mergedGossip)
mergedGossip.nodes.find(_.address == m2.address).get.timestamp must be(m2Updated.timestamp)
val mergedGossip = g1 merge g2
mergedGossip.nodes.map(_.address) must be(Set(m1.address, m2.address, m3.address))
mergedGossip.nodeMetricsFor(m1.address).map(_.metrics) must be(Some(m1.metrics))
mergedGossip.nodeMetricsFor(m2.address).map(_.metrics) must be(Some(m2Updated.metrics))
mergedGossip.nodeMetricsFor(m3.address).map(_.metrics) must be(Some(m3.metrics))
mergedGossip.nodes.foreach(_.metrics.size must be > (3))
mergedGossip.nodeMetricsFor(m2.address).map(_.timestamp) must be(Some(m2Updated.timestamp))
}
"get the current NodeMetrics if it exists in the local nodes" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
var localGossip = MetricsGossip(window)
localGossip :+= m1
localGossip.metricsFor(m1).nonEmpty must be(true)
val g1 = MetricsGossip.empty :+ m1
g1.nodeMetricsFor(m1.address).map(_.metrics) must be(Some(m1.metrics))
}
"remove a node if it is no longer Up" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
var localGossip = MetricsGossip(window)
localGossip :+= m1
localGossip :+= m2
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size must be(2)
val g2 = g1 remove m1.address
g2.nodes.size must be(1)
g2.nodes.exists(_.address == m1.address) must be(false)
g2.nodeMetricsFor(m1.address) must be(None)
g2.nodeMetricsFor(m2.address).map(_.metrics) must be(Some(m2.metrics))
}
localGossip.nodes.size must be(2)
localGossip = localGossip remove m1.address
localGossip.nodes.size must be(1)
localGossip.nodes.exists(_.address == m1.address) must be(false)
"filter nodes" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size must be(2)
val g2 = g1 filter Set(m2.address)
g2.nodes.size must be(1)
g2.nodes.exists(_.address == m1.address) must be(false)
g2.nodeMetricsFor(m1.address) must be(None)
g2.nodeMetricsFor(m2.address).map(_.metrics) must be(Some(m2.metrics))
}
}
}

View file

@ -4,51 +4,44 @@
package akka.cluster
import akka.testkit.AkkaSpec
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.actor.Address
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class NodeMetricsSpec extends AkkaSpec with AbstractClusterMetricsSpec with MetricSpec {
val collector = createMetricsCollector
class NodeMetricsSpec extends WordSpec with MustMatchers {
val node1 = Address("akka", "sys", "a", 2554)
val node2 = Address("akka", "sys", "a", 2555)
"NodeMetrics must" must {
"recognize updatable nodes" in {
(NodeMetrics(node1, 0) updatable NodeMetrics(node1, 1)) must be(true)
}
"recognize non-updatable nodes" in {
(NodeMetrics(node1, 1) updatable NodeMetrics(node2, 0)) must be(false)
}
"return correct result for 2 'same' nodes" in {
(NodeMetrics(node1, 0) same NodeMetrics(node1, 0)) must be(true)
(NodeMetrics(node1, 0) sameAs NodeMetrics(node1, 0)) must be(true)
}
"return correct result for 2 not 'same' nodes" in {
(NodeMetrics(node1, 0) same NodeMetrics(node2, 0)) must be(false)
(NodeMetrics(node1, 0) sameAs NodeMetrics(node2, 0)) must be(false)
}
"merge 2 NodeMetrics by most recent" in {
val sample1 = NodeMetrics(node1, 1, collector.sample.metrics)
val sample2 = NodeMetrics(node1, 2, collector.sample.metrics)
val sample1 = NodeMetrics(node1, 1, Set(Metric.create("a", 10, None), Metric.create("b", 20, None)).flatten)
val sample2 = NodeMetrics(node1, 2, Set(Metric.create("a", 11, None), Metric.create("c", 30, None)).flatten)
val merged = sample1 merge sample2
merged.timestamp must be(sample2.timestamp)
merged.metrics must be(sample2.metrics)
merged.metric("a").map(_.value) must be(Some(11))
merged.metric("b").map(_.value) must be(Some(20))
merged.metric("c").map(_.value) must be(Some(30))
}
"not merge 2 NodeMetrics if master is more recent" in {
val sample1 = NodeMetrics(node1, 1, collector.sample.metrics)
val sample2 = NodeMetrics(node2, 0, sample1.metrics)
val sample1 = NodeMetrics(node1, 1, Set(Metric.create("a", 10, None), Metric.create("b", 20, None)).flatten)
val sample2 = NodeMetrics(node1, 0, Set(Metric.create("a", 11, None), Metric.create("c", 30, None)).flatten)
val merged = sample2 merge sample2 // older and not same
merged.timestamp must be(sample2.timestamp)
merged.metrics must be(sample2.metrics)
val merged = sample1 merge sample2 // older and not same
merged.timestamp must be(sample1.timestamp)
merged.metrics must be(sample1.metrics)
}
}
}

View file

@ -0,0 +1,118 @@
/*
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.routing
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.actor.Address
import akka.cluster.Metric
import akka.cluster.NodeMetrics
import akka.cluster.StandardMetrics._
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsSelectorSpec extends WordSpec with MustMatchers {
val abstractSelector = new CapacityMetricsSelector {
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = Map.empty
}
val a1 = Address("akka", "sys", "a1", 2551)
val b1 = Address("akka", "sys", "b1", 2551)
val c1 = Address("akka", "sys", "c1", 2551)
val d1 = Address("akka", "sys", "d1", 2551)
val decayFactor = Some(0.18)
val nodeMetricsA = NodeMetrics(a1, System.currentTimeMillis, Set(
Metric.create(HeapMemoryUsed, 128, decayFactor),
Metric.create(HeapMemoryCommitted, 256, decayFactor),
Metric.create(HeapMemoryMax, 512, None),
Metric.create(CpuCombined, 0.1, decayFactor),
Metric.create(SystemLoadAverage, 0.5, None),
Metric.create(Processors, 8, None)).flatten)
val nodeMetricsB = NodeMetrics(b1, System.currentTimeMillis, Set(
Metric.create(HeapMemoryUsed, 256, decayFactor),
Metric.create(HeapMemoryCommitted, 512, decayFactor),
Metric.create(HeapMemoryMax, 1024, None),
Metric.create(CpuCombined, 0.5, decayFactor),
Metric.create(SystemLoadAverage, 1.0, None),
Metric.create(Processors, 16, None)).flatten)
val nodeMetricsC = NodeMetrics(c1, System.currentTimeMillis, Set(
Metric.create(HeapMemoryUsed, 1024, decayFactor),
Metric.create(HeapMemoryCommitted, 1024, decayFactor),
Metric.create(HeapMemoryMax, 1024, None),
Metric.create(CpuCombined, 1.0, decayFactor),
Metric.create(SystemLoadAverage, 16.0, None),
Metric.create(Processors, 16, None)).flatten)
val nodeMetricsD = NodeMetrics(d1, System.currentTimeMillis, Set(
Metric.create(HeapMemoryUsed, 511, decayFactor),
Metric.create(HeapMemoryCommitted, 512, decayFactor),
Metric.create(HeapMemoryMax, 512, None),
Metric.create(Processors, 2, decayFactor)).flatten)
val nodeMetrics = Set(nodeMetricsA, nodeMetricsB, nodeMetricsC, nodeMetricsD)
"CapacityMetricsSelector" must {
"calculate weights from capacity" in {
val capacity = Map(a1 -> 0.6, b1 -> 0.3, c1 -> 0.1)
val weights = abstractSelector.weights(capacity)
weights must be(Map(c1 -> 1, b1 -> 3, a1 -> 6))
}
"handle low and zero capacity" in {
val capacity = Map(a1 -> 0.0, b1 -> 1.0, c1 -> 0.005, d1 -> 0.004)
val weights = abstractSelector.weights(capacity)
weights must be(Map(a1 -> 0, b1 -> 100, c1 -> 1, d1 -> 0))
}
}
"HeapMetricsSelector" must {
"calculate capacity of heap metrics" in {
val capacity = HeapMetricsSelector.capacity(nodeMetrics)
capacity(a1) must be(0.75 plusOrMinus 0.0001)
capacity(b1) must be(0.75 plusOrMinus 0.0001)
capacity(c1) must be(0.0 plusOrMinus 0.0001)
capacity(d1) must be(0.001953125 plusOrMinus 0.0001)
}
}
"CpuMetricsSelector" must {
"calculate capacity of cpuCombined metrics" in {
val capacity = CpuMetricsSelector.capacity(nodeMetrics)
capacity(a1) must be(0.9 plusOrMinus 0.0001)
capacity(b1) must be(0.5 plusOrMinus 0.0001)
capacity(c1) must be(0.0 plusOrMinus 0.0001)
capacity.contains(d1) must be(false)
}
}
"SystemLoadAverageMetricsSelector" must {
"calculate capacity of systemLoadAverage metrics" in {
val capacity = SystemLoadAverageMetricsSelector.capacity(nodeMetrics)
capacity(a1) must be(0.9375 plusOrMinus 0.0001)
capacity(b1) must be(0.9375 plusOrMinus 0.0001)
capacity(c1) must be(0.0 plusOrMinus 0.0001)
capacity.contains(d1) must be(false)
}
}
"MixMetricsSelector" must {
"aggregate capacity of all metrics" in {
val capacity = MixMetricsSelector.capacity(nodeMetrics)
capacity(a1) must be((0.75 + 0.9 + 0.9375) / 3 plusOrMinus 0.0001)
capacity(b1) must be((0.75 + 0.5 + 0.9375) / 3 plusOrMinus 0.0001)
capacity(c1) must be((0.0 + 0.0 + 0.0) / 3 plusOrMinus 0.0001)
capacity(d1) must be((0.001953125) / 1 plusOrMinus 0.0001)
}
}
}

View file

@ -0,0 +1,87 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.routing
import com.typesafe.config.ConfigFactory
import akka.actor.Address
import akka.actor.RootActorPath
import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.netty.port = 0
""")) {
val a1 = Address("akka", "sys", "a1", 2551)
val b1 = Address("akka", "sys", "b1", 2551)
val c1 = Address("akka", "sys", "c1", 2551)
val d1 = Address("akka", "sys", "d1", 2551)
val refA = system.actorFor(RootActorPath(a1) / "user" / "a")
val refB = system.actorFor(RootActorPath(b1) / "user" / "b")
val refC = system.actorFor(RootActorPath(c1) / "user" / "c")
"WeightedRoutees" must {
"allocate weighted refs" in {
val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10)
val refs = Vector(refA, refB, refC)
val weighted = new WeightedRoutees(refs, a1, weights)
weighted(1) must be(refA)
2 to 4 foreach { weighted(_) must be(refB) }
5 to 14 foreach { weighted(_) must be(refC) }
weighted.total must be(14)
}
"check boundaries" in {
val empty = new WeightedRoutees(Vector(), a1, Map.empty)
empty.isEmpty must be(true)
intercept[IllegalArgumentException] {
empty.total
}
val weighted = new WeightedRoutees(Vector(refA, refB, refC), a1, Map.empty)
weighted.total must be(3)
intercept[IllegalArgumentException] {
weighted(0)
}
intercept[IllegalArgumentException] {
weighted(4)
}
}
"allocate refs for undefined weight" in {
val weights = Map(a1 -> 1, b1 -> 7)
val refs = Vector(refA, refB, refC)
val weighted = new WeightedRoutees(refs, a1, weights)
weighted(1) must be(refA)
2 to 8 foreach { weighted(_) must be(refB) }
// undefined, uses the mean of the weights, i.e. 4
9 to 12 foreach { weighted(_) must be(refC) }
weighted.total must be(12)
}
"allocate weighted local refs" in {
val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10)
val refs = Vector(testActor, refB, refC)
val weighted = new WeightedRoutees(refs, a1, weights)
1 to 2 foreach { weighted(_) must be(testActor) }
3 to weighted.total foreach { weighted(_) must not be (testActor) }
}
"not allocate ref with weight zero" in {
val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10)
val refs = Vector(refA, refB, refC)
val weighted = new WeightedRoutees(refs, a1, weights)
1 to weighted.total foreach { weighted(_) must not be (refA) }
}
}
}

Some files were not shown because too many files have changed in this diff Show more