Merge remote-tracking branch 'origin/master' into wip-1644-programmatic-deploy-∂π

This commit is contained in:
Roland 2012-02-03 09:49:04 +01:00
commit 45140b465e
306 changed files with 14713 additions and 11320 deletions

View file

@ -14,6 +14,7 @@ import java.util.LinkedList;
import java.lang.Iterable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import static akka.japi.Util.manifest;
import akka.testkit.AkkaSpec;
@ -45,7 +46,7 @@ public class JavaFutureTests {
}
}, system.dispatcher());
Future<String> f2 = f1.map(new Function<String, String>() {
Future<String> f2 = f1.map(new Mapper<String, String>() {
public String apply(String s) {
return s + " World";
}
@ -59,8 +60,8 @@ public class JavaFutureTests {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
f.onSuccess(new Procedure<String>() {
public void apply(String result) {
f.onSuccess(new OnSuccess<String>() {
public void onSuccess(String result) {
if (result.equals("foo"))
latch.countDown();
}
@ -76,8 +77,8 @@ public class JavaFutureTests {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
f.onFailure(new Procedure<Throwable>() {
public void apply(Throwable t) {
f.onFailure(new OnFailure() {
public void onFailure(Throwable t) {
if (t instanceof NullPointerException)
latch.countDown();
}
@ -94,8 +95,8 @@ public class JavaFutureTests {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
f.onComplete(new Procedure2<Throwable,String>() {
public void apply(Throwable t, String r) {
f.onComplete(new OnComplete<String>() {
public void onComplete(Throwable t, String r) {
latch.countDown();
}
});
@ -110,8 +111,8 @@ public class JavaFutureTests {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
f.foreach(new Procedure<String>() {
public void apply(String future) {
f.foreach(new Foreach<String>() {
public void each(String future) {
latch.countDown();
}
});
@ -127,7 +128,7 @@ public class JavaFutureTests {
Promise<String> cf = Futures.promise(system.dispatcher());
cf.success("1000");
Future<String> f = cf;
Future<Integer> r = f.flatMap(new Function<String, Future<Integer>>() {
Future<Integer> r = f.flatMap(new Mapper<String, Future<Integer>>() {
public Future<Integer> apply(String r) {
latch.countDown();
Promise<Integer> cf = Futures.promise(system.dispatcher());
@ -146,8 +147,8 @@ public class JavaFutureTests {
final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(system.dispatcher());
Future<String> f = cf;
Future<String> r = f.filter(new Function<String, Boolean>() {
public Boolean apply(String r) {
Future<String> r = f.filter(new Filter<String>() {
public boolean filter(String r) {
latch.countDown();
return r.equals("foo");
}
@ -267,15 +268,55 @@ public class JavaFutureTests {
}
}, system.dispatcher());
assertEquals(expect, Await.result(f, timeout));
assertEquals(expect, Await.result(f, timeout).get());
}
@Test
public void BlockMustBeCallable() {
public void blockMustBeCallable() {
Promise<String> p = Futures.promise(system.dispatcher());
Duration d = Duration.create(1, TimeUnit.SECONDS);
p.success("foo");
Await.ready(p, d);
assertEquals(Await.result(p, d), "foo");
}
@Test
public void mapToMustBeCallable() {
Promise<Object> p = Futures.promise(system.dispatcher());
Future<String> f = p.future().mapTo(manifest(String.class));
Duration d = Duration.create(1, TimeUnit.SECONDS);
p.success("foo");
Await.ready(p, d);
assertEquals(Await.result(p, d), "foo");
}
@Test
public void recoverToMustBeCallable() {
final IllegalStateException fail = new IllegalStateException("OHNOES");
Promise<Object> p = Futures.promise(system.dispatcher());
Future<Object> f = p.future().recover(new Recover<Object>() {
public Object recover(Throwable t) throws Throwable {
if (t == fail) return "foo";
else throw t;
}
});
Duration d = Duration.create(1, TimeUnit.SECONDS);
p.failure(fail);
assertEquals(Await.result(f, d), "foo");
}
@Test
public void recoverWithToMustBeCallable() {
final IllegalStateException fail = new IllegalStateException("OHNOES");
Promise<Object> p = Futures.promise(system.dispatcher());
Future<Object> f = p.future().recoverWith(new Recover<Future<Object>>() {
public Future<Object> recover(Throwable t) throws Throwable {
if (t == fail) return Futures.<Object>successful("foo", system.dispatcher()).future();
else throw t;
}
});
Duration d = Duration.create(1, TimeUnit.SECONDS);
p.failure(fail);
assertEquals(Await.result(f, d), "foo");
}
}

View file

@ -3,53 +3,30 @@
*/
package akka.actor
import org.scalatest.BeforeAndAfterAll
import akka.util.duration._
import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout
import java.util.concurrent.TimeoutException
import akka.testkit._
import akka.dispatch.Await
import akka.util.Timeout
import akka.pattern.{ ask, AskTimeoutException }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeout {
class ActorTimeoutSpec extends AkkaSpec {
val defaultTimeout = system.settings.ActorTimeout.duration
val testTimeout = if (system.settings.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
val testTimeout = 200.millis.dilated
"An Actor-based Future" must {
"use the global default timeout if no implicit in scope" in {
within(defaultTimeout - 100.millis, defaultTimeout + 400.millis) {
val echo = system.actorOf(Props.empty)
try {
val d = system.settings.ActorTimeout.duration
val f = echo ? "hallo"
intercept[AskTimeoutException] { Await.result(f, d + d) }
} finally { system.stop(echo) }
}
}
"use implicitly supplied timeout" in {
implicit val timeout = Timeout(testTimeout)
within(testTimeout - 100.millis, testTimeout + 300.millis) {
val echo = system.actorOf(Props.empty)
try {
val f = (echo ? "hallo").mapTo[String]
intercept[AskTimeoutException] { Await.result(f, testTimeout + testTimeout) }
} finally { system.stop(echo) }
}
val f = (echo ? "hallo")
intercept[AskTimeoutException] { Await.result(f, testTimeout * 2) }
}
"use explicitly supplied timeout" in {
within(testTimeout - 100.millis, testTimeout + 300.millis) {
val echo = system.actorOf(Props.empty)
val f = echo.?("hallo")(testTimeout)
try {
intercept[AskTimeoutException] { Await.result(f, testTimeout + 300.millis) }
} finally { system.stop(echo) }
}
intercept[AskTimeoutException] { Await.result(f, testTimeout * 2) }
}
}
}

View file

@ -9,6 +9,8 @@ object ConsistencySpec {
consistency-dispatcher {
throughput = 1
keep-alive-time = 1 ms
executor = "thread-pool-executor"
thread-pool-executor {
core-pool-size-min = 10
core-pool-size-max = 10
max-pool-size-min = 10
@ -16,6 +18,7 @@ object ConsistencySpec {
task-queue-type = array
task-queue-size = 7
}
}
"""
class CacheMisaligned(var value: Long, var padding1: Long, var padding2: Long, var padding3: Int) //Vars, no final fences

View file

@ -7,7 +7,6 @@ package akka.actor
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.testkit._
import TestEvent.Mute
import FSM._
import akka.util.duration._
import akka.event._
import com.typesafe.config.ConfigFactory
@ -52,7 +51,7 @@ object FSMActorSpec {
}
}
case Event("hello", _) stay replying "world"
case Event("bye", _) stop(Shutdown)
case Event("bye", _) stop(FSM.Shutdown)
}
when(Open) {
@ -63,7 +62,7 @@ object FSMActorSpec {
}
whenUnhandled {
case Ev(msg) {
case Event(msg, _) {
log.warning("unhandled event " + msg + " in state " + stateName + " with data " + stateData)
unhandledLatch.open
stay
@ -82,7 +81,7 @@ object FSMActorSpec {
}
onTermination {
case StopEvent(Shutdown, Locked, _)
case StopEvent(FSM.Shutdown, Locked, _)
// stop is called from lockstate with shutdown as reason...
terminatedLatch.open
}
@ -110,6 +109,8 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
"unlock the lock" in {
import FSM.{ Transition, CurrentState, SubscribeTransitionCallBack }
val latches = new Latches
import latches._
@ -163,7 +164,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
val fsm = TestActorRef(new Actor with FSM[Int, Null] {
startWith(1, null)
when(1) {
case Ev("go") goto(2)
case Event("go", _) goto(2)
}
})
val name = fsm.path.toString
@ -182,7 +183,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
lazy val fsm = new Actor with FSM[Int, Null] {
override def preStart = { started.countDown }
startWith(1, null)
when(1) { NullFunction }
when(1) { FSM.NullFunction }
onTermination {
case x testActor ! x
}
@ -190,7 +191,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
val ref = system.actorOf(Props(fsm))
Await.ready(started, timeout.duration)
system.stop(ref)
expectMsg(1 second, fsm.StopEvent(Shutdown, 1, null))
expectMsg(1 second, fsm.StopEvent(FSM.Shutdown, 1, null))
}
"log events and transitions if asked to do so" in {
@ -204,12 +205,12 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
val fsm = TestActorRef(new Actor with LoggingFSM[Int, Null] {
startWith(1, null)
when(1) {
case Ev("go")
setTimer("t", Shutdown, 1.5 seconds, false)
case Event("go", _)
setTimer("t", FSM.Shutdown, 1.5 seconds, false)
goto(2)
}
when(2) {
case Ev("stop")
case Event("stop", _)
cancelTimer("t")
stop
}
@ -230,7 +231,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
expectMsgPF(1 second, hint = "processing Event(stop,null)") {
case Logging.Debug(`name`, `fsmClass`, s: String) if s.startsWith("processing Event(stop,null) from Actor[") true
}
expectMsgAllOf(1 second, Logging.Debug(name, fsmClass, "canceling timer 't'"), Normal)
expectMsgAllOf(1 second, Logging.Debug(name, fsmClass, "canceling timer 't'"), FSM.Normal)
expectNoMsg(1 second)
system.eventStream.unsubscribe(testActor)
}
@ -251,6 +252,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
})
fsmref ! "log"
val fsm = fsmref.underlyingActor
import FSM.LogEntry
expectMsg(1 second, IndexedSeq(LogEntry(1, 0, "log")))
fsmref ! "count"
fsmref ! "log"

View file

@ -160,37 +160,37 @@ object FSMTimingSpec {
startWith(Initial, 0)
when(Initial) {
case Ev(TestSingleTimer)
case Event(TestSingleTimer, _)
setTimer("tester", Tick, 500 millis, false)
goto(TestSingleTimer)
case Ev(TestRepeatedTimer)
case Event(TestRepeatedTimer, _)
setTimer("tester", Tick, 100 millis, true)
goto(TestRepeatedTimer) using 4
case Ev(TestStateTimeoutOverride)
case Event(TestStateTimeoutOverride, _)
goto(TestStateTimeout) forMax (Duration.Inf)
case Ev(x: FSMTimingSpec.State) goto(x)
case Event(x: FSMTimingSpec.State, _) goto(x)
}
when(TestStateTimeout, stateTimeout = 500 millis) {
case Ev(StateTimeout) goto(Initial)
case Ev(Cancel) goto(Initial) replying (Cancel)
case Event(StateTimeout, _) goto(Initial)
case Event(Cancel, _) goto(Initial) replying (Cancel)
}
when(TestSingleTimer) {
case Ev(Tick)
case Event(Tick, _)
tester ! Tick
goto(Initial)
}
when(TestCancelTimer) {
case Ev(Tick)
case Event(Tick, _)
setTimer("hallo", Tock, 1 milli, false)
TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1 second)
cancelTimer("hallo")
sender ! Tick
setTimer("hallo", Tock, 500 millis, false)
stay
case Ev(Tock)
case Event(Tock, _)
tester ! Tock
stay
case Ev(Cancel)
case Event(Cancel, _)
cancelTimer("hallo")
goto(Initial)
}
@ -206,29 +206,29 @@ object FSMTimingSpec {
}
when(TestCancelStateTimerInNamedTimerMessage) {
// FSM is suspended after processing this message and resumed 500ms later
case Ev(Tick)
case Event(Tick, _)
suspend(self)
setTimer("named", Tock, 1 millis, false)
TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1 second)
stay forMax (1 millis) replying Tick
case Ev(Tock)
case Event(Tock, _)
goto(TestCancelStateTimerInNamedTimerMessage2)
}
when(TestCancelStateTimerInNamedTimerMessage2) {
case Ev(StateTimeout)
case Event(StateTimeout, _)
goto(Initial)
case Ev(Cancel)
case Event(Cancel, _)
goto(Initial) replying Cancel
}
when(TestUnhandled) {
case Ev(SetHandler)
case Event(SetHandler, _)
whenUnhandled {
case Ev(Tick)
case Event(Tick, _)
tester ! Unhandled(Tick)
stay
}
stay
case Ev(Cancel)
case Event(Cancel, _)
whenUnhandled(NullFunction)
goto(Initial)
}

View file

@ -5,7 +5,6 @@ package akka.actor
import akka.testkit._
import akka.util.duration._
import FSM._
import akka.util.Duration
object FSMTransitionSpec {
@ -17,13 +16,13 @@ object FSMTransitionSpec {
class MyFSM(target: ActorRef) extends Actor with FSM[Int, Unit] {
startWith(0, Unit)
when(0) {
case Ev("tick") goto(1)
case Event("tick", _) goto(1)
}
when(1) {
case Ev("tick") goto(0)
case Event("tick", _) goto(0)
}
whenUnhandled {
case Ev("reply") stay replying "reply"
case Event("reply", _) stay replying "reply"
}
initialize
override def preRestart(reason: Throwable, msg: Option[Any]) { target ! "restarted" }
@ -32,10 +31,10 @@ object FSMTransitionSpec {
class OtherFSM(target: ActorRef) extends Actor with FSM[Int, Int] {
startWith(0, 0)
when(0) {
case Ev("tick") goto(1) using (1)
case Event("tick", _) goto(1) using (1)
}
when(1) {
case Ev(_) stay
case _ stay
}
onTransition {
case 0 -> 1 target ! ((stateData, nextStateData))
@ -56,6 +55,8 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
"A FSM transition notifier" must {
"notify listeners" in {
import FSM.{ SubscribeTransitionCallBack, CurrentState, Transition }
val fsm = system.actorOf(Props(new MyFSM(testActor)))
within(1 second) {
fsm ! SubscribeTransitionCallBack(testActor)
@ -77,8 +78,8 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
}))
within(300 millis) {
fsm ! SubscribeTransitionCallBack(forward)
expectMsg(CurrentState(fsm, 0))
fsm ! FSM.SubscribeTransitionCallBack(forward)
expectMsg(FSM.CurrentState(fsm, 0))
system.stop(forward)
fsm ! "tick"
expectNoMsg

View file

@ -4,7 +4,7 @@
package akka.actor
import akka.util.{ ByteString, Duration, Timer }
import akka.util.{ ByteString, Duration, Deadline }
import akka.util.duration._
import scala.util.continuations._
import akka.testkit._
@ -244,13 +244,13 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout {
val promise = Promise[T]()(executor)
val timer = timeout match {
case Some(duration) Some(Timer(duration))
val timer: Option[Deadline] = timeout match {
case Some(duration) Some(duration fromNow)
case None None
}
def check(n: Int, e: Throwable): Boolean =
(count.isEmpty || (n < count.get)) && (timer.isEmpty || timer.get.isTicking) && (filter.isEmpty || filter.get(e))
(count.isEmpty || (n < count.get)) && (timer.isEmpty || timer.get.hasTimeLeft()) && (filter.isEmpty || filter.get(e))
def run(n: Int) {
future onComplete {

View file

@ -14,11 +14,14 @@ object LocalActorRefProviderSpec {
akka {
actor {
default-dispatcher {
executor = "thread-pool-executor"
thread-pool-executor {
core-pool-size-min = 16
core-pool-size-max = 16
}
}
}
}
"""
}

View file

@ -25,11 +25,14 @@ object TypedActorSpec {
val config = """
pooled-dispatcher {
type = BalancingDispatcher
executor = "thread-pool-executor"
thread-pool-executor {
core-pool-size-min = 60
core-pool-size-max = 60
max-pool-size-min = 60
max-pool-size-max = 60
}
}
"""
class CyclicIterator[T](val items: Seq[T]) extends Iterator[T] {

View file

@ -448,16 +448,14 @@ object DispatcherModelSpec {
class MessageDispatcherInterceptorConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(config, prerequisites) {
private val instance: MessageDispatcher = {
configureThreadPool(config,
threadPoolConfig new Dispatcher(prerequisites,
private val instance: MessageDispatcher =
new Dispatcher(prerequisites,
config.getString("id"),
config.getInt("throughput"),
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType,
threadPoolConfig,
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) with MessageDispatcherInterceptor).build
}
configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) with MessageDispatcherInterceptor
override def dispatcher(): MessageDispatcher = instance
}
@ -522,16 +520,14 @@ object BalancingDispatcherModelSpec {
class BalancingMessageDispatcherInterceptorConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(config, prerequisites) {
private val instance: MessageDispatcher = {
configureThreadPool(config,
threadPoolConfig new BalancingDispatcher(prerequisites,
private val instance: MessageDispatcher =
new BalancingDispatcher(prerequisites,
config.getString("id"),
config.getInt("throughput"),
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType,
threadPoolConfig,
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) with MessageDispatcherInterceptor).build
}
configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) with MessageDispatcherInterceptor
override def dispatcher(): MessageDispatcher = instance
}

View file

@ -16,15 +16,21 @@ object DispatcherActorSpec {
}
test-throughput-dispatcher {
throughput = 101
executor = "thread-pool-executor"
thread-pool-executor {
core-pool-size-min = 1
core-pool-size-max = 1
}
}
test-throughput-deadline-dispatcher {
throughput = 2
throughput-deadline-time = 100 milliseconds
executor = "thread-pool-executor"
thread-pool-executor {
core-pool-size-min = 1
core-pool-size-max = 1
}
}
"""
class TestActor extends Actor {

View file

@ -18,27 +18,14 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
val settings = system.settings
val config = settings.config
{
import config._
getString("akka.version") must equal("2.0-SNAPSHOT")
settings.ConfigVersion must equal("2.0-SNAPSHOT")
getBoolean("akka.daemonic") must equal(false)
getString("akka.actor.default-dispatcher.type") must equal("Dispatcher")
getMilliseconds("akka.actor.default-dispatcher.keep-alive-time") must equal(60 * 1000)
getDouble("akka.actor.default-dispatcher.core-pool-size-factor") must equal(3.0)
getDouble("akka.actor.default-dispatcher.max-pool-size-factor") must equal(3.0)
getInt("akka.actor.default-dispatcher.task-queue-size") must equal(-1)
getString("akka.actor.default-dispatcher.task-queue-type") must equal("linked")
getBoolean("akka.actor.default-dispatcher.allow-core-timeout") must equal(true)
getInt("akka.actor.default-dispatcher.mailbox-capacity") must equal(-1)
getMilliseconds("akka.actor.default-dispatcher.mailbox-push-timeout-time") must equal(10 * 1000)
getString("akka.actor.default-dispatcher.mailboxType") must be("")
getMilliseconds("akka.actor.default-dispatcher.shutdown-timeout") must equal(1 * 1000)
getInt("akka.actor.default-dispatcher.throughput") must equal(5)
getMilliseconds("akka.actor.default-dispatcher.throughput-deadline-time") must equal(0)
getBoolean("akka.actor.serialize-messages") must equal(false)
settings.SerializeAllMessages must equal(false)
@ -48,5 +35,45 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
getMilliseconds("akka.scheduler.tickDuration") must equal(100)
settings.SchedulerTickDuration must equal(100 millis)
}
{
val c = config.getConfig("akka.actor.default-dispatcher")
//General dispatcher config
{
c.getString("type") must equal("Dispatcher")
c.getString("executor") must equal("fork-join-executor")
c.getInt("mailbox-capacity") must equal(-1)
c.getMilliseconds("mailbox-push-timeout-time") must equal(10 * 1000)
c.getString("mailboxType") must be("")
c.getMilliseconds("shutdown-timeout") must equal(1 * 1000)
c.getInt("throughput") must equal(5)
c.getMilliseconds("throughput-deadline-time") must equal(0)
}
//Fork join executor config
{
val pool = c.getConfig("fork-join-executor")
pool.getInt("parallelism-min") must equal(8)
pool.getDouble("parallelism-factor") must equal(3.0)
pool.getInt("parallelism-max") must equal(64)
}
//Thread pool executor config
{
val pool = c.getConfig("thread-pool-executor")
import pool._
getMilliseconds("keep-alive-time") must equal(60 * 1000)
getDouble("core-pool-size-factor") must equal(3.0)
getDouble("max-pool-size-factor") must equal(3.0)
getInt("task-queue-size") must equal(-1)
getString("task-queue-type") must equal("linked")
getBoolean("allow-core-timeout") must equal(true)
}
}
}
}
}

View file

@ -5,11 +5,10 @@ package akka.dataflow
import akka.actor.{ Actor, Props }
import akka.dispatch.{ Future, Await }
import akka.actor.future2actor
import akka.util.duration._
import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout
import akka.pattern.ask
import akka.pattern.{ ask, pipe }
class Future2ActorSpec extends AkkaSpec with DefaultTimeout {

View file

@ -13,10 +13,10 @@ import akka.testkit.AkkaSpec
import org.scalatest.junit.JUnitSuite
import akka.testkit.DefaultTimeout
import akka.testkit.TestLatch
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch }
import scala.runtime.NonLocalReturnControl
import akka.pattern.ask
import java.lang.{ IllegalStateException, ArithmeticException }
import java.util.concurrent._
object FutureSpec {
class TestActor extends Actor {
@ -39,7 +39,6 @@ object FutureSpec {
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class JavaFutureSpec extends JavaFutureTests with JUnitSuite
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
@ -55,11 +54,11 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
val empty = Promise[String]()
val timedOut = Promise.successful[String]("Timedout")
Await.result(failure or timedOut, timeout.duration) must be("Timedout")
Await.result(timedOut or empty, timeout.duration) must be("Timedout")
Await.result(failure or failure or timedOut, timeout.duration) must be("Timedout")
Await.result(failure fallbackTo timedOut, timeout.duration) must be("Timedout")
Await.result(timedOut fallbackTo empty, timeout.duration) must be("Timedout")
Await.result(failure fallbackTo failure fallbackTo timedOut, timeout.duration) must be("Timedout")
intercept[RuntimeException] {
Await.result(failure or otherFailure, timeout.duration)
Await.result(failure fallbackTo otherFailure, timeout.duration)
}.getMessage must be("last")
}
}
@ -303,6 +302,32 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
}
}
"recoverWith from exceptions" in {
val o = new IllegalStateException("original")
val r = new IllegalStateException("recovered")
intercept[IllegalStateException] {
Await.result(Promise.failed[String](o) recoverWith { case _ if false == true Promise.successful("yay!") }, timeout.duration)
} must be(o)
Await.result(Promise.failed[String](o) recoverWith { case _ Promise.successful("yay!") }, timeout.duration) must equal("yay!")
intercept[IllegalStateException] {
Await.result(Promise.failed[String](o) recoverWith { case _ Promise.failed[String](r) }, timeout.duration)
} must be(r)
}
"andThen like a boss" in {
val q = new LinkedBlockingQueue[Int]
for (i 1 to 1000) {
Await.result(Future { q.add(1); 3 } andThen { case _ q.add(2) } andThen { case Right(0) q.add(Int.MaxValue) } andThen { case _ q.add(3); }, timeout.duration) must be(3)
q.poll() must be(1)
q.poll() must be(2)
q.poll() must be(3)
q.clear()
}
}
"firstCompletedOf" in {
val futures = Vector.fill[Future[Int]](10)(Promise[Int]()) :+ Promise.successful[Int](5)
Await.result(Future.firstCompletedOf(futures), timeout.duration) must be(5)
@ -856,7 +881,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
"be completed" in { f((future, _) future must be('completed)) }
"contain a value" in { f((future, result) future.value must be(Some(Right(result)))) }
"return result with 'get'" in { f((future, result) Await.result(future, timeout.duration) must be(result)) }
"return result with 'Await.sync'" in { f((future, result) Await.result(future, timeout.duration) must be(result)) }
"return result with 'Await.result'" in { f((future, result) Await.result(future, timeout.duration) must be(result)) }
"not timeout" in { f((future, _) Await.ready(future, 0 millis)) }
"filter result" in {
f { (future, result)
@ -907,7 +932,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
})
}
"throw exception with 'get'" in { f((future, message) (evaluating { Await.result(future, timeout.duration) } must produce[E]).getMessage must be(message)) }
"throw exception with 'Await.sync'" in { f((future, message) (evaluating { Await.result(future, timeout.duration) } must produce[E]).getMessage must be(message)) }
"throw exception with 'Await.result'" in { f((future, message) (evaluating { Await.result(future, timeout.duration) } must produce[E]).getMessage must be(message)) }
"retain exception with filter" in {
f { (future, message)
(evaluating { Await.result(future filter (_ true), timeout.duration) } must produce[E]).getMessage must be(message)

View file

@ -1,169 +0,0 @@
package akka.performance.microbench
import akka.performance.workbench.PerformanceSpec
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import akka.actor._
import java.util.concurrent.{ ThreadPoolExecutor, CountDownLatch, TimeUnit }
import akka.dispatch._
import java.util.concurrent.ThreadPoolExecutor.AbortPolicy
import java.util.concurrent.BlockingQueue
import java.util.concurrent.LinkedBlockingQueue
import akka.util.Duration
import akka.util.duration._
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TellThroughput10000PerformanceSpec extends PerformanceSpec {
import TellThroughput10000PerformanceSpec._
val repeat = 30000L * repeatFactor
"Tell" must {
"warmup" in {
runScenario(4, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
}
"perform with load 1" in {
runScenario(1)
}
"perform with load 2" in {
runScenario(2)
}
"perform with load 4" in {
runScenario(4)
}
"perform with load 6" in {
runScenario(6)
}
"perform with load 8" in {
runScenario(8)
}
"perform with load 10" in {
runScenario(10)
}
"perform with load 12" in {
runScenario(12)
}
"perform with load 14" in {
runScenario(14)
}
"perform with load 16" in {
runScenario(16)
}
"perform with load 18" in {
runScenario(18)
}
"perform with load 20" in {
runScenario(20)
}
"perform with load 22" in {
runScenario(22)
}
"perform with load 24" in {
runScenario(24)
}
"perform with load 26" in {
runScenario(26)
}
"perform with load 28" in {
runScenario(28)
}
"perform with load 30" in {
runScenario(30)
}
"perform with load 32" in {
runScenario(32)
}
"perform with load 34" in {
runScenario(34)
}
"perform with load 36" in {
runScenario(36)
}
"perform with load 38" in {
runScenario(38)
}
"perform with load 40" in {
runScenario(40)
}
"perform with load 42" in {
runScenario(42)
}
"perform with load 44" in {
runScenario(44)
}
"perform with load 46" in {
runScenario(46)
}
"perform with load 48" in {
runScenario(48)
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val dispatcherKey = "benchmark.high-throughput-dispatcher"
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients)
yield system.actorOf(Props(new Destination).withDispatcher(dispatcherKey))
val clients = for ((dest, j) destinations.zipWithIndex)
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(dispatcherKey))
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(system.stop(_))
destinations.foreach(system.stop(_))
}
}
}
}
object TellThroughput10000PerformanceSpec {
case object Run
case object Msg
class Destination extends Actor {
def receive = {
case Msg sender ! Msg
}
}
class Client(
actor: ActorRef,
latch: CountDownLatch,
repeat: Long) extends Actor {
var sent = 0L
var received = 0L
def receive = {
case Msg
received += 1
if (sent < repeat) {
actor ! Msg
sent += 1
} else if (received >= repeat) {
latch.countDown()
}
case Run
for (i 0L until math.min(20000L, repeat)) {
actor ! Msg
sent += 1
}
}
}
}

View file

@ -100,15 +100,14 @@ class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val clientDispatcher = "benchmark.client-dispatcher"
val destinationDispatcher = "benchmark.destination-dispatcher"
val throughputDispatcher = "benchmark.throughput-dispatcher"
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients)
yield system.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
yield system.actorOf(Props(new Destination).withDispatcher(throughputDispatcher))
val clients = for (dest destinations)
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(throughputDispatcher))
val start = System.nanoTime
clients.foreach(_ ! Run)

View file

@ -16,10 +16,10 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
"Tell" must {
"warmup" in {
runScenario(4, warmup = true)
runScenario(8, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
runScenario(8, warmup = true)
}
"perform with load 1" in {
runScenario(1)
@ -48,19 +48,66 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
"perform with load 16" in {
runScenario(16)
}
"perform with load 18" in {
runScenario(18)
}
"perform with load 20" in {
runScenario(20)
}
"perform with load 22" in {
runScenario(22)
}
"perform with load 24" in {
runScenario(24)
}
"perform with load 26" in {
runScenario(26)
}
"perform with load 28" in {
runScenario(28)
}
"perform with load 30" in {
runScenario(30)
}
"perform with load 32" in {
runScenario(32)
}
"perform with load 34" in {
runScenario(34)
}
"perform with load 36" in {
runScenario(36)
}
"perform with load 38" in {
runScenario(38)
}
"perform with load 40" in {
runScenario(40)
}
"perform with load 42" in {
runScenario(42)
}
"perform with load 44" in {
runScenario(44)
}
"perform with load 46" in {
runScenario(46)
}
"perform with load 48" in {
runScenario(48)
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val clientDispatcher = "benchmark.client-dispatcher"
val destinationDispatcher = "benchmark.destination-dispatcher"
val throughputDispatcher = "benchmark.throughput-dispatcher"
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients)
yield system.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
yield system.actorOf(Props(new Destination).withDispatcher(throughputDispatcher))
val clients = for (dest destinations)
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(throughputDispatcher))
val start = System.nanoTime
clients.foreach(_ ! Run)

View file

@ -1,171 +0,0 @@
package akka.performance.microbench
import akka.performance.workbench.PerformanceSpec
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import akka.actor._
import java.util.concurrent.{ ThreadPoolExecutor, CountDownLatch, TimeUnit }
import akka.dispatch._
import java.util.concurrent.ThreadPoolExecutor.AbortPolicy
import java.util.concurrent.BlockingQueue
import java.util.concurrent.LinkedBlockingQueue
import akka.util.Duration
import akka.util.duration._
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TellThroughputPinnedDispatchersPerformanceSpec extends PerformanceSpec {
import TellThroughputPinnedDispatchersPerformanceSpec._
val repeat = 30000L * repeatFactor
"Tell" must {
"warmup" in {
runScenario(4, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
}
"perform with load 1" in {
runScenario(1)
}
"perform with load 2" in {
runScenario(2)
}
"perform with load 4" in {
runScenario(4)
}
"perform with load 6" in {
runScenario(6)
}
"perform with load 8" in {
runScenario(8)
}
"perform with load 10" in {
runScenario(10)
}
"perform with load 12" in {
runScenario(12)
}
"perform with load 14" in {
runScenario(14)
}
"perform with load 16" in {
runScenario(16)
}
"perform with load 18" in {
runScenario(18)
}
"perform with load 20" in {
runScenario(20)
}
"perform with load 22" in {
runScenario(22)
}
"perform with load 24" in {
runScenario(24)
}
"perform with load 26" in {
runScenario(26)
}
"perform with load 28" in {
runScenario(28)
}
"perform with load 30" in {
runScenario(30)
}
"perform with load 32" in {
runScenario(32)
}
"perform with load 34" in {
runScenario(34)
}
"perform with load 36" in {
runScenario(36)
}
"perform with load 38" in {
runScenario(38)
}
"perform with load 40" in {
runScenario(40)
}
"perform with load 42" in {
runScenario(42)
}
"perform with load 44" in {
runScenario(44)
}
"perform with load 46" in {
runScenario(46)
}
"perform with load 48" in {
runScenario(48)
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val pinnedDispatcher = "benchmark.pinned-dispatcher"
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients)
yield system.actorOf(Props(new Destination).withDispatcher(pinnedDispatcher))
val clients = for ((dest, j) destinations.zipWithIndex)
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(pinnedDispatcher))
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(system.stop(_))
destinations.foreach(system.stop(_))
}
}
}
}
object TellThroughputPinnedDispatchersPerformanceSpec {
case object Run
case object Msg
class Destination extends Actor {
def receive = {
case Msg sender ! Msg
}
}
class Client(
actor: ActorRef,
latch: CountDownLatch,
repeat: Long) extends Actor {
var sent = 0L
var received = 0L
def receive = {
case Msg
received += 1
if (sent < repeat) {
actor ! Msg
sent += 1
} else if (received >= repeat) {
latch.countDown()
}
case Run
for (i 0L until math.min(1000L, repeat)) {
actor ! Msg
sent += 1
}
}
}
}

View file

@ -84,7 +84,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
} yield Bid(s + i, 100 - i, 1000)
val orders = askOrders.zip(bidOrders).map(x Seq(x._1, x._2)).flatten
val clientDispatcher = "benchmark.client-dispatcher"
val latencyDispatcher = "benchmark.trading-dispatcher"
val ordersPerClient = repeat * orders.size / numberOfClients
val totalNumberOfOrders = ordersPerClient * numberOfClients
@ -93,7 +93,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelay.toMicros.toInt)).withDispatcher(clientDispatcher)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelay.toMicros.toInt)).withDispatcher(latencyDispatcher)
system.actorOf(props)
})

View file

@ -39,11 +39,9 @@ class AkkaTradingSystem(val system: ActorSystem) extends TradingSystem {
val orDispatcher = orderReceiverDispatcher
val meDispatcher = matchingEngineDispatcher
// by default we use default-dispatcher
def orderReceiverDispatcher: Option[String] = None
def orderReceiverDispatcher: Option[String] = Some("benchmark.trading-dispatcher")
// by default we use default-dispatcher
def matchingEngineDispatcher: Option[String] = None
def matchingEngineDispatcher: Option[String] = Some("benchmark.trading-dispatcher")
override val orderbooksGroupedByMatchingEngine: List[List[Orderbook]] =
for (groupOfSymbols: List[String] OrderbookRepository.orderbookSymbolsGroupedByMatchingEngine)

View file

@ -81,7 +81,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
} yield Bid(s + i, 100 - i, 1000)
val orders = askOrders.zip(bidOrders).map(x Seq(x._1, x._2)).flatten
val clientDispatcher = "benchmark.client-dispatcher"
val throughputDispatcher = "benchmark.trading-dispatcher"
val ordersPerClient = repeat * orders.size / numberOfClients
val totalNumberOfOrders = ordersPerClient * numberOfClients
@ -90,7 +90,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher)
val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(throughputDispatcher)
system.actorOf(props)
})

View file

@ -20,38 +20,40 @@ object BenchmarkConfig {
resultDir = "target/benchmark"
useDummyOrderbook = false
client-dispatcher {
core-pool-size-min = ${benchmark.maxClients}
core-pool-size-max = ${benchmark.maxClients}
throughput-dispatcher {
throughput = 5
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = ${benchmark.maxClients}
parallelism-max = ${benchmark.maxClients}
}
destination-dispatcher {
core-pool-size-min = ${benchmark.maxClients}
core-pool-size-max = ${benchmark.maxClients}
}
high-throughput-dispatcher {
throughput = 10000
core-pool-size-min = ${benchmark.maxClients}
core-pool-size-max = ${benchmark.maxClients}
}
pinned-dispatcher {
type = PinnedDispatcher
}
latency-dispatcher {
throughput = 1
core-pool-size-min = ${benchmark.maxClients}
core-pool-size-max = ${benchmark.maxClients}
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = ${benchmark.maxClients}
parallelism-max = ${benchmark.maxClients}
}
}
trading-dispatcher {
throughput = 5
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = ${benchmark.maxClients}
parallelism-max = ${benchmark.maxClients}
}
}
}
""")
private val longRunningBenchmarkConfig = ConfigFactory.parseString("""
benchmark {
longRunning = true
minClients = 4
maxClients = 48
repeatFactor = 150
repeatFactor = 2000
maxRunDuration = 120 seconds
useDummyOrderbook = true
}

View file

@ -31,7 +31,8 @@ abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends Akk
def compareResultWith: Option[String] = None
def acceptClients(numberOfClients: Int): Boolean = {
(minClients <= numberOfClients && numberOfClients <= maxClients)
(minClients <= numberOfClients && numberOfClients <= maxClients &&
(maxClients <= 16 || numberOfClients % 4 == 0))
}
def logMeasurement(numberOfClients: Int, durationNs: Long, n: Long) {

View file

@ -20,9 +20,12 @@ object ConfiguredLocalRoutingSpec {
akka {
actor {
default-dispatcher {
executor = "thread-pool-executor"
thread-pool-executor {
core-pool-size-min = 8
core-pool-size-max = 16
}
}
deployment {
/config {
router = random

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,119 @@
/*
* Written by Doug Lea with assistance from members of JCP JSR-166
* Expert Group and released to the public domain, as explained at
* http://creativecommons.org/publicdomain/zero/1.0/
*/
package akka.jsr166y;
/**
* A thread managed by a {@link ForkJoinPool}, which executes
* {@link ForkJoinTask}s.
* This class is subclassable solely for the sake of adding
* functionality -- there are no overridable methods dealing with
* scheduling or execution. However, you can override initialization
* and termination methods surrounding the main task processing loop.
* If you do create such a subclass, you will also need to supply a
* custom {@link ForkJoinPool.ForkJoinWorkerThreadFactory} to use it
* in a {@code ForkJoinPool}.
*
* @since 1.7
* @author Doug Lea
*/
public class ForkJoinWorkerThread extends Thread {
/*
* ForkJoinWorkerThreads are managed by ForkJoinPools and perform
* ForkJoinTasks. For explanation, see the internal documentation
* of class ForkJoinPool.
*/
final ForkJoinPool.WorkQueue workQueue; // Work-stealing mechanics
final ForkJoinPool pool; // the pool this thread works in
/**
* Creates a ForkJoinWorkerThread operating in the given pool.
*
* @param pool the pool this thread works in
* @throws NullPointerException if pool is null
*/
protected ForkJoinWorkerThread(ForkJoinPool pool) {
super(pool.nextWorkerName());
setDaemon(true);
Thread.UncaughtExceptionHandler ueh = pool.ueh;
if (ueh != null)
setUncaughtExceptionHandler(ueh);
this.pool = pool;
this.workQueue = new ForkJoinPool.WorkQueue(this, pool.localMode);
pool.registerWorker(this);
}
/**
* Returns the pool hosting this thread.
*
* @return the pool
*/
public ForkJoinPool getPool() {
return pool;
}
/**
* Returns the index number of this thread in its pool. The
* returned value ranges from zero to the maximum number of
* threads (minus one) that have ever been created in the pool.
* This method may be useful for applications that track status or
* collect results per-worker rather than per-task.
*
* @return the index number
*/
public int getPoolIndex() {
return workQueue.poolIndex;
}
/**
* Initializes internal state after construction but before
* processing any tasks. If you override this method, you must
* invoke {@code super.onStart()} at the beginning of the method.
* Initialization requires care: Most fields must have legal
* default values, to ensure that attempted accesses from other
* threads work correctly even before this thread starts
* processing tasks.
*/
protected void onStart() {
}
/**
* Performs cleanup associated with termination of this worker
* thread. If you override this method, you must invoke
* {@code super.onTermination} at the end of the overridden method.
*
* @param exception the exception causing this thread to abort due
* to an unrecoverable error, or {@code null} if completed normally
*/
protected void onTermination(Throwable exception) {
}
/**
* This method is required to be public, but should never be
* called explicitly. It performs the main run loop to execute
* {@link ForkJoinTask}s.
*/
public void run() {
Throwable exception = null;
try {
onStart();
pool.runWorker(this);
} catch (Throwable ex) {
exception = ex;
} finally {
try {
onTermination(exception);
} catch (Throwable ex) {
if (exception == null)
exception = ex;
} finally {
pool.deregisterWorker(this, exception);
}
}
}
}

View file

@ -0,0 +1,164 @@
/*
* Written by Doug Lea with assistance from members of JCP JSR-166
* Expert Group and released to the public domain, as explained at
* http://creativecommons.org/publicdomain/zero/1.0/
*/
package akka.jsr166y;
/**
* A recursive resultless {@link ForkJoinTask}. This class
* establishes conventions to parameterize resultless actions as
* {@code Void} {@code ForkJoinTask}s. Because {@code null} is the
* only valid value of type {@code Void}, methods such as {@code join}
* always return {@code null} upon completion.
*
* <p><b>Sample Usages.</b> Here is a simple but complete ForkJoin
* sort that sorts a given {@code long[]} array:
*
* <pre> {@code
* static class SortTask extends RecursiveAction {
* final long[] array; final int lo, hi;
* SortTask(long[] array, int lo, int hi) {
* this.array = array; this.lo = lo; this.hi = hi;
* }
* SortTask(long[] array) { this(array, 0, array.length); }
* protected void compute() {
* if (hi - lo < THRESHOLD)
* sortSequentially(lo, hi);
* else {
* int mid = (lo + hi) >>> 1;
* invokeAll(new SortTask(array, lo, mid),
* new SortTask(array, mid, hi));
* merge(lo, mid, hi);
* }
* }
* // implementation details follow:
* final static int THRESHOLD = 1000;
* void sortSequentially(int lo, int hi) {
* Arrays.sort(array, lo, hi);
* }
* void merge(int lo, int mid, int hi) {
* long[] buf = Arrays.copyOfRange(array, lo, mid);
* for (int i = 0, j = lo, k = mid; i < buf.length; j++)
* array[j] = (k == hi || buf[i] < array[k]) ?
* buf[i++] : array[k++];
* }
* }}</pre>
*
* You could then sort {@code anArray} by creating {@code new
* SortTask(anArray)} and invoking it in a ForkJoinPool. As a more
* concrete simple example, the following task increments each element
* of an array:
* <pre> {@code
* class IncrementTask extends RecursiveAction {
* final long[] array; final int lo, hi;
* IncrementTask(long[] array, int lo, int hi) {
* this.array = array; this.lo = lo; this.hi = hi;
* }
* protected void compute() {
* if (hi - lo < THRESHOLD) {
* for (int i = lo; i < hi; ++i)
* array[i]++;
* }
* else {
* int mid = (lo + hi) >>> 1;
* invokeAll(new IncrementTask(array, lo, mid),
* new IncrementTask(array, mid, hi));
* }
* }
* }}</pre>
*
* <p>The following example illustrates some refinements and idioms
* that may lead to better performance: RecursiveActions need not be
* fully recursive, so long as they maintain the basic
* divide-and-conquer approach. Here is a class that sums the squares
* of each element of a double array, by subdividing out only the
* right-hand-sides of repeated divisions by two, and keeping track of
* them with a chain of {@code next} references. It uses a dynamic
* threshold based on method {@code getSurplusQueuedTaskCount}, but
* counterbalances potential excess partitioning by directly
* performing leaf actions on unstolen tasks rather than further
* subdividing.
*
* <pre> {@code
* double sumOfSquares(ForkJoinPool pool, double[] array) {
* int n = array.length;
* Applyer a = new Applyer(array, 0, n, null);
* pool.invoke(a);
* return a.result;
* }
*
* class Applyer extends RecursiveAction {
* final double[] array;
* final int lo, hi;
* double result;
* Applyer next; // keeps track of right-hand-side tasks
* Applyer(double[] array, int lo, int hi, Applyer next) {
* this.array = array; this.lo = lo; this.hi = hi;
* this.next = next;
* }
*
* double atLeaf(int l, int h) {
* double sum = 0;
* for (int i = l; i < h; ++i) // perform leftmost base step
* sum += array[i] * array[i];
* return sum;
* }
*
* protected void compute() {
* int l = lo;
* int h = hi;
* Applyer right = null;
* while (h - l > 1 && getSurplusQueuedTaskCount() <= 3) {
* int mid = (l + h) >>> 1;
* right = new Applyer(array, mid, h, right);
* right.fork();
* h = mid;
* }
* double sum = atLeaf(l, h);
* while (right != null) {
* if (right.tryUnfork()) // directly calculate if not stolen
* sum += right.atLeaf(right.lo, right.hi);
* else {
* right.join();
* sum += right.result;
* }
* right = right.next;
* }
* result = sum;
* }
* }}</pre>
*
* @since 1.7
* @author Doug Lea
*/
public abstract class RecursiveAction extends ForkJoinTask<Void> {
private static final long serialVersionUID = 5232453952276485070L;
/**
* The main computation performed by this task.
*/
protected abstract void compute();
/**
* Always returns {@code null}.
*
* @return {@code null} always
*/
public final Void getRawResult() { return null; }
/**
* Requires null completion value.
*/
protected final void setRawResult(Void mustBeNull) { }
/**
* Implements execution conventions for RecursiveActions.
*/
protected final boolean exec() {
compute();
return true;
}
}

View file

@ -0,0 +1,68 @@
/*
* Written by Doug Lea with assistance from members of JCP JSR-166
* Expert Group and released to the public domain, as explained at
* http://creativecommons.org/publicdomain/zero/1.0/
*/
package akka.jsr166y;
/**
* A recursive result-bearing {@link ForkJoinTask}.
*
* <p>For a classic example, here is a task computing Fibonacci numbers:
*
* <pre> {@code
* class Fibonacci extends RecursiveTask<Integer> {
* final int n;
* Fibonacci(int n) { this.n = n; }
* Integer compute() {
* if (n <= 1)
* return n;
* Fibonacci f1 = new Fibonacci(n - 1);
* f1.fork();
* Fibonacci f2 = new Fibonacci(n - 2);
* return f2.compute() + f1.join();
* }
* }}</pre>
*
* However, besides being a dumb way to compute Fibonacci functions
* (there is a simple fast linear algorithm that you'd use in
* practice), this is likely to perform poorly because the smallest
* subtasks are too small to be worthwhile splitting up. Instead, as
* is the case for nearly all fork/join applications, you'd pick some
* minimum granularity size (for example 10 here) for which you always
* sequentially solve rather than subdividing.
*
* @since 1.7
* @author Doug Lea
*/
public abstract class RecursiveTask<V> extends ForkJoinTask<V> {
private static final long serialVersionUID = 5232453952276485270L;
/**
* The result of the computation.
*/
V result;
/**
* The main computation performed by this task.
*/
protected abstract V compute();
public final V getRawResult() {
return result;
}
protected final void setRawResult(V value) {
result = value;
}
/**
* Implements execution conventions for RecursiveTask.
*/
protected final boolean exec() {
result = compute();
return true;
}
}

View file

@ -32,8 +32,6 @@ import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import org.omg.CORBA.portable.IDLEntity;
import com.eaio.util.lang.Hex;
/**

View file

@ -1,86 +0,0 @@
package com.eaio.uuid;
/**
* com/eaio/uuid/UUIDHelper.java .
* Generated by the IDL-to-Java compiler (portable), version "3.1"
* from uuid.idl
* Sonntag, 7. März 2004 21.35 Uhr CET
*/
/**
* The UUID struct.
*/
abstract public class UUIDHelper
{
private static String _id = "IDL:com/eaio/uuid/UUID:1.0";
public static void insert (org.omg.CORBA.Any a, com.eaio.uuid.UUID that)
{
org.omg.CORBA.portable.OutputStream out = a.create_output_stream ();
a.type (type ());
write (out, that);
a.read_value (out.create_input_stream (), type ());
}
public static com.eaio.uuid.UUID extract (org.omg.CORBA.Any a)
{
return read (a.create_input_stream ());
}
private static org.omg.CORBA.TypeCode __typeCode = null;
private static boolean __active = false;
synchronized public static org.omg.CORBA.TypeCode type ()
{
if (__typeCode == null)
{
synchronized (org.omg.CORBA.TypeCode.class)
{
if (__typeCode == null)
{
if (__active)
{
return org.omg.CORBA.ORB.init().create_recursive_tc ( _id );
}
__active = true;
org.omg.CORBA.StructMember[] _members0 = new org.omg.CORBA.StructMember [2];
org.omg.CORBA.TypeCode _tcOf_members0 = null;
_tcOf_members0 = org.omg.CORBA.ORB.init ().get_primitive_tc (org.omg.CORBA.TCKind.tk_longlong);
_members0[0] = new org.omg.CORBA.StructMember (
"time",
_tcOf_members0,
null);
_tcOf_members0 = org.omg.CORBA.ORB.init ().get_primitive_tc (org.omg.CORBA.TCKind.tk_longlong);
_members0[1] = new org.omg.CORBA.StructMember (
"clockSeqAndNode",
_tcOf_members0,
null);
__typeCode = org.omg.CORBA.ORB.init ().create_struct_tc (com.eaio.uuid.UUIDHelper.id (), "UUID", _members0);
__active = false;
}
}
}
return __typeCode;
}
public static String id ()
{
return _id;
}
public static com.eaio.uuid.UUID read (org.omg.CORBA.portable.InputStream istream)
{
com.eaio.uuid.UUID value = new com.eaio.uuid.UUID ();
value.time = istream.read_longlong ();
value.clockSeqAndNode = istream.read_longlong ();
return value;
}
public static void write (org.omg.CORBA.portable.OutputStream ostream, com.eaio.uuid.UUID value)
{
ostream.write_longlong (value.time);
ostream.write_longlong (value.clockSeqAndNode);
}
}

View file

@ -1,42 +0,0 @@
package com.eaio.uuid;
/**
* com/eaio/uuid/UUIDHolder.java .
* Generated by the IDL-to-Java compiler (portable), version "3.1"
* from uuid.idl
* Sonntag, 7. März 2004 21.35 Uhr CET
*/
/**
* The UUID struct.
*/
public final class UUIDHolder implements org.omg.CORBA.portable.Streamable
{
public com.eaio.uuid.UUID value = null;
public UUIDHolder ()
{
}
public UUIDHolder (com.eaio.uuid.UUID initialValue)
{
value = initialValue;
}
public void _read (org.omg.CORBA.portable.InputStream i)
{
value = com.eaio.uuid.UUIDHelper.read (i);
}
public void _write (org.omg.CORBA.portable.OutputStream o)
{
com.eaio.uuid.UUIDHelper.write (o, value);
}
public org.omg.CORBA.TypeCode _type ()
{
return com.eaio.uuid.UUIDHelper.type ();
}
}

View file

@ -159,16 +159,38 @@ akka {
# parameters
type = "Dispatcher"
# Which kind of ExecutorService to use for this dispatcher
# Valid options:
# "fork-join-executor" requires a "fork-join-executor" section
# "thread-pool-executor" requires a "thread-pool-executor" section
# or
# A FQCN of a class extending ExecutorServiceConfigurator
executor = "fork-join-executor"
# This will be used if you have set "executor = "fork-join-executor""
fork-join-executor {
# Min number of threads to cap factor-based parallelism number to
parallelism-min = 8
# Parallelism (threads) ... ceil(available processors * factor)
parallelism-factor = 3.0
# Max number of threads to cap factor-based parallelism number to
parallelism-max = 64
}
# This will be used if you have set "executor = "thread-pool-executor""
thread-pool-executor {
# Keep alive time for threads
keep-alive-time = 60s
# minimum number of threads to cap factor-based core number to
# Min number of threads to cap factor-based core number to
core-pool-size-min = 8
# No of core threads ... ceil(available processors * factor)
core-pool-size-factor = 3.0
# maximum number of threads to cap factor-based number to
# Max number of threads to cap factor-based number to
core-pool-size-max = 64
# Hint: max-pool-size is only used for bounded task queues
@ -178,7 +200,7 @@ akka {
# Max no of threads ... ceil(available processors * factor)
max-pool-size-factor = 3.0
# maximum number of threads to cap factor-based max number to
# Max number of threads to cap factor-based max number to
max-pool-size-max = 64
# Specifies the bounded capacity of the task queue (< 1 == unbounded)
@ -190,6 +212,7 @@ akka {
# Allow core threads to time out
allow-core-timeout = on
}
# How long time the dispatcher will wait for new actors until it shuts down
shutdown-timeout = 1s

View file

@ -7,6 +7,7 @@ package akka.actor
import akka.AkkaException
import scala.reflect.BeanProperty
import scala.util.control.NoStackTrace
import scala.collection.immutable.Stack
import java.util.regex.Pattern
/**
@ -112,6 +113,7 @@ object Actor {
def isDefinedAt(x: Any) = false
def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()")
}
}
/**
@ -172,7 +174,7 @@ trait Actor {
type Receive = Actor.Receive
/**
* Stores the context for this actor, including self, sender, and hotswap.
* Stores the context for this actor, including self, and sender.
* It is implicit to support operations such as `forward`.
*
* [[akka.actor.ActorContext]] is the Scala API. `getContext` returns a
@ -281,15 +283,37 @@ trait Actor {
// ==== INTERNAL IMPLEMENTATION DETAILS ====
// =========================================
/**
* For Akka internal use only.
*/
private[akka] final def apply(msg: Any) = {
val behaviorStack = context.asInstanceOf[ActorCell].hotswap
msg match {
case msg if behaviorStack.nonEmpty && behaviorStack.head.isDefinedAt(msg) behaviorStack.head.apply(msg)
case msg if behaviorStack.isEmpty && processingBehavior.isDefinedAt(msg) processingBehavior.apply(msg)
case unknown unhandled(unknown)
}
// TODO would it be more efficient to assume that most messages are matched and catch MatchError instead of using isDefinedAt?
val head = behaviorStack.head
if (head.isDefinedAt(msg)) head.apply(msg) else unhandled(msg)
}
private[this] val processingBehavior = receive //ProcessingBehavior is the original behavior
/**
* For Akka internal use only.
*/
private[akka] def pushBehavior(behavior: Receive): Unit = {
behaviorStack = behaviorStack.push(behavior)
}
/**
* For Akka internal use only.
*/
private[akka] def popBehavior(): Unit = {
val original = behaviorStack
val popped = original.pop
behaviorStack = if (popped.isEmpty) original else popped
}
/**
* For Akka internal use only.
*/
private[akka] def clearBehaviorStack(): Unit =
behaviorStack = Stack.empty[Receive].push(behaviorStack.last)
private var behaviorStack: Stack[Receive] = Stack.empty[Receive].push(receive)
}

View file

@ -174,8 +174,7 @@ private[akka] class ActorCell(
val self: InternalActorRef,
val props: Props,
@volatile var parent: InternalActorRef,
/*no member*/ _receiveTimeout: Option[Duration],
var hotswap: Stack[PartialFunction[Any, Unit]]) extends UntypedActorContext {
/*no member*/ _receiveTimeout: Option[Duration]) extends UntypedActorContext {
import ActorCell._
@ -209,10 +208,10 @@ private[akka] class ActorCell(
/**
* In milliseconds
*/
final var receiveTimeoutData: (Long, Cancellable) =
var receiveTimeoutData: (Long, Cancellable) =
if (_receiveTimeout.isDefined) (_receiveTimeout.get.toMillis, emptyCancellable) else emptyReceiveTimeoutData
final var childrenRefs: TreeMap[String, ChildRestartStats] = emptyChildrenRefs
var childrenRefs: TreeMap[String, ChildRestartStats] = emptyChildrenRefs
private def _actorOf(props: Props, name: String): ActorRef = {
if (system.settings.SerializeAllCreators && !props.creator.isInstanceOf[NoSerializationVerificationNeeded]) {
@ -255,16 +254,16 @@ private[akka] class ActorCell(
a.stop()
}
final var currentMessage: Envelope = null
var currentMessage: Envelope = null
final var actor: Actor = _
var actor: Actor = _
final var stopping = false
var stopping = false
@volatile //This must be volatile since it isn't protected by the mailbox status
var mailbox: Mailbox = _
final var nextNameSequence: Long = 0
var nextNameSequence: Long = 0
//Not thread safe, so should only be used inside the actor that inhabits this ActorCell
final protected def randomName(): String = {
@ -389,7 +388,6 @@ private[akka] class ActorCell(
}
}
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
hotswap = Props.noHotSwap // Reset the behavior
freshActor.postRestart(cause)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(freshActor), "restarted"))
@ -509,9 +507,9 @@ private[akka] class ActorCell(
}
}
def become(behavior: Actor.Receive, discardOld: Boolean = true) {
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
if (discardOld) unbecome()
hotswap = hotswap.push(behavior)
actor.pushBehavior(behavior)
}
/**
@ -527,10 +525,7 @@ private[akka] class ActorCell(
become(newReceive, discardOld)
}
def unbecome() {
val h = hotswap
if (h.nonEmpty) hotswap = h.pop
}
def unbecome(): Unit = actor.popBehavior()
def autoReceiveMessage(msg: Envelope) {
if (system.settings.DebugAutoReceive)
@ -547,9 +542,9 @@ private[akka] class ActorCell(
}
private def doTerminate() {
try {
try {
val a = actor
try {
try {
if (a ne null) a.postStop()
} finally {
dispatcher.detach(this)
@ -563,7 +558,7 @@ private[akka] class ActorCell(
} finally {
currentMessage = null
clearActorFields()
hotswap = Props.noHotSwap
if (a ne null) a.clearBehaviorStack()
}
}
}

View file

@ -57,7 +57,7 @@ import akka.event.LoggingAdapter
*
* } else if (o instanceof Request3) {
* val msg = ((Request3) o).getMsg();
* getSender().tell(other.ask(msg, 5000)); // reply with Future for holding the others reply (timeout 5 seconds)
* getSender().tell(ask(other, msg, 5000)); // reply with Future for holding the others reply (timeout 5 seconds)
*
* } else {
* unhandled(o);
@ -224,8 +224,7 @@ private[akka] class LocalActorRef private[akka] (
_supervisor: InternalActorRef,
val path: ActorPath,
val systemService: Boolean = false,
_receiveTimeout: Option[Duration] = None,
_hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap)
_receiveTimeout: Option[Duration] = None)
extends InternalActorRef with LocalRef {
/*
@ -238,7 +237,7 @@ private[akka] class LocalActorRef private[akka] (
* us to use purely factory methods for creating LocalActorRefs.
*/
@volatile
private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout, _hotswap)
private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout)
actorCell.start()
protected def newActorCell(
@ -246,9 +245,8 @@ private[akka] class LocalActorRef private[akka] (
ref: InternalActorRef,
props: Props,
supervisor: InternalActorRef,
receiveTimeout: Option[Duration],
hotswap: Stack[PartialFunction[Any, Unit]]): ActorCell =
new ActorCell(system, ref, props, supervisor, receiveTimeout, hotswap)
receiveTimeout: Option[Duration]): ActorCell =
new ActorCell(system, ref, props, supervisor, receiveTimeout)
protected def actorContext: ActorContext = actorCell

View file

View file

@ -1,6 +1,7 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import akka.config.ConfigurationException

View file

@ -48,6 +48,14 @@ object FSM {
}
}
/**
* This extractor is just convenience for matching a (S, S) pair, including a
* reminder what the new state is.
*/
object -> {
def unapply[S](in: (S, S)) = Some(in)
}
case class LogEntry[S, D](stateName: S, stateData: D, event: Any)
case class State[S, D](stateName: S, stateData: D, timeout: Option[Duration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) {
@ -174,6 +182,10 @@ trait FSM[S, D] extends Listeners {
type Timeout = Option[Duration]
type TransitionHandler = PartialFunction[(S, S), Unit]
// import so that it is visible without an import
val -> = FSM.->
val StateTimeout = FSM.StateTimeout
val log = Logging(context.system, this)
/**
@ -284,14 +296,6 @@ trait FSM[S, D] extends Listeners {
*/
protected final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout
/**
* This extractor is just convenience for matching a (S, S) pair, including a
* reminder what the new state is.
*/
object -> {
def unapply[S](in: (S, S)) = Some(in)
}
/**
* 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
@ -533,9 +537,6 @@ trait FSM[S, D] extends Listeners {
}
case class Event(event: Any, stateData: D)
object Ev {
def unapply[D](e: Event): Option[Any] = Some(e.event)
}
case class StopEvent[S, D](reason: Reason, currentState: S, stateData: D)
}

View file

@ -47,36 +47,36 @@ case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int =
trait SupervisorStrategyLowPriorityImplicits { this: SupervisorStrategy.type
/**
* Implicit conversion from `Seq` of Cause-Action pairs to a `Decider`. See makeDecider(causeAction).
* Implicit conversion from `Seq` of Cause-Directive pairs to a `Decider`. See makeDecider(causeDirective).
*/
implicit def seqCauseAction2Decider(trapExit: Iterable[CauseAction]): Decider = makeDecider(trapExit)
implicit def seqCauseDirective2Decider(trapExit: Iterable[CauseDirective]): Decider = makeDecider(trapExit)
// the above would clash with seqThrowable2Decider for empty lists
}
object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
sealed trait Action
sealed trait Directive
/**
* Resumes message processing for the failed Actor
*/
case object Resume extends Action
case object Resume extends Directive
/**
* Discards the old Actor instance and replaces it with a new,
* then resumes message processing.
*/
case object Restart extends Action
case object Restart extends Directive
/**
* Stops the Actor
*/
case object Stop extends Action
case object Stop extends Directive
/**
* Escalates the failure to the supervisor of the supervisor,
* by rethrowing the cause of the failure.
*/
case object Escalate extends Action
case object Escalate extends Directive
/**
* Resumes message processing for the failed Actor
@ -127,9 +127,9 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
*/
implicit def seqThrowable2Decider(trapExit: Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit)
type Decider = PartialFunction[Throwable, Action]
type JDecider = akka.japi.Function[Throwable, Action]
type CauseAction = (Class[_ <: Throwable], Action)
type Decider = PartialFunction[Throwable, Directive]
type JDecider = akka.japi.Function[Throwable, Directive]
type CauseDirective = (Class[_ <: Throwable], Directive)
/**
* Decider builder which just checks whether one of
@ -152,14 +152,14 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(trapExit.toSeq)
/**
* Decider builder for Iterables of cause-action pairs, e.g. a map obtained
* Decider builder for Iterables of cause-directive pairs, e.g. a map obtained
* from configuration; will sort the pairs so that the most specific type is
* checked before all its subtypes, allowing carving out subtrees of the
* Throwable hierarchy.
*/
def makeDecider(flat: Iterable[CauseAction]): Decider = {
val actions = sort(flat)
return { case x actions find (_._1 isInstance x) map (_._2) getOrElse Escalate }
def makeDecider(flat: Iterable[CauseDirective]): Decider = {
val directives = sort(flat)
return { case x directives find (_._1 isInstance x) map (_._2) getOrElse Escalate }
}
def makeDecider(func: JDecider): Decider = {
@ -170,8 +170,8 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
* Sort so that subtypes always precede their supertypes, but without
* obeying any order between unrelated subtypes (insert sort).
*/
def sort(in: Iterable[CauseAction]): Seq[CauseAction] =
(new ArrayBuffer[CauseAction](in.size) /: in) { (buf, ca)
def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca)
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
case -1 buf append ca
case x buf insert (x, ca)
@ -215,8 +215,8 @@ abstract class SupervisorStrategy {
* Returns whether it processed the failure or not
*/
def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = {
val action = if (decider.isDefinedAt(cause)) decider(cause) else Escalate
action match {
val directive = if (decider.isDefinedAt(cause)) decider(cause) else Escalate
directive match {
case Resume child.asInstanceOf[InternalActorRef].resume(); true
case Restart processFailure(context, true, child, cause, stats, children); true
case Stop processFailure(context, false, child, cause, stats, children); true
@ -227,10 +227,13 @@ abstract class SupervisorStrategy {
}
/**
* Restart all child actors when one fails
* Applies the fault handling `Directive` (Resume, Restart, Stop) specified in the `Decider`
* to all children when one fails, as opposed to [[akka.actor.OneForOneStrategy]] that applies
* it only to the child actor that failed.
*
* @param maxNrOfRetries the number of times an actor is allowed to be restarted, negative value means no limit
* @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window
* @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Action]], you can also use a
* @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a
* `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates.
*/
case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider)
@ -270,10 +273,13 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
}
/**
* Restart a child actor when it fails
* Applies the fault handling `Directive` (Resume, Restart, Stop) specified in the `Decider`
* to the child actor that failed, as opposed to [[akka.actor.AllForOneStrategy]] that applies
* it to all children.
*
* @param maxNrOfRetries the number of times an actor is allowed to be restarted, negative value means no limit
* @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window
* @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Action]], you can also use a
* @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a
* `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates.
*/
case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider)

View file

@ -24,7 +24,6 @@ object Props {
final val defaultDeploy = Deploy()
final val noHotSwap: Stack[Actor.Receive] = Stack.empty
final val empty = new Props(() new Actor { def receive = Actor.emptyBehavior })
/**

View file

@ -1,15 +1,7 @@
/*
* Copyright 2007 WorldWide Conferencing, LLC
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import akka.util.Duration
@ -134,7 +126,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
receiver ! message
// Check if the receiver is still alive and kicking before reschedule the task
if (receiver.isTerminated) {
log.warning("Could not reschedule message to be sent because receiving actor has been terminated.")
log.debug("Could not reschedule message to be sent because receiving actor has been terminated.")
} else {
scheduleNext(timeout, delay, continuousCancellable)
}

View file

@ -37,9 +37,9 @@ import akka.japi.{ Creator }
* }
*
* private static SupervisorStrategy strategy = new OneForOneStrategy(10, Duration.parse("1 minute"),
* new Function<Throwable, Action>() {
* new Function<Throwable, Directive>() {
* @Override
* public Action apply(Throwable t) {
* public Directive apply(Throwable t) {
* if (t instanceof ArithmeticException) {
* return resume();
* } else if (t instanceof NullPointerException) {

View file

@ -27,15 +27,4 @@ package object actor {
val i = n.lastIndexOf('.')
n.substring(i + 1)
}
implicit def future2actor[T](f: akka.dispatch.Future[T]) = new {
def pipeTo(actor: ActorRef): this.type = {
f onComplete {
case Right(r) actor ! r
case Left(f) actor ! Status.Failure(f)
}
this
}
}
}

View file

@ -14,6 +14,7 @@ import akka.event.EventStream
import com.typesafe.config.Config
import akka.util.ReflectiveAccess
import akka.serialization.SerializationExtension
import akka.jsr166y.ForkJoinPool
final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) {
if (message.isInstanceOf[AnyRef]) {
@ -292,6 +293,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
protected[akka] def shutdown(): Unit
}
abstract class ExecutorServiceConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceFactoryProvider
/**
* Base class to be used for hooking in new dispatchers into Dispatchers.
*/
@ -333,14 +336,30 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
}
}
def configureThreadPool(
config: Config,
createDispatcher: (ThreadPoolConfig) MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
import ThreadPoolConfigDispatcherBuilder.conf_?
def configureExecutor(): ExecutorServiceConfigurator = {
config.getString("executor") match {
case null | "" | "fork-join-executor" new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites)
case "thread-pool-executor" new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
case fqcn
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites])
ReflectiveAccess.createInstance[ExecutorServiceConfigurator](fqcn, constructorSignature, Array[AnyRef](config, prerequisites), prerequisites.classloader) match {
case Right(instance) instance
case Left(exception) throw new IllegalArgumentException(
("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s],
make sure it has an accessible constructor with a [%s,%s] signature""")
.format(fqcn, config.getString("id"), classOf[Config], classOf[DispatcherPrerequisites]), exception)
}
}
}
}
//Apply the following options to the config if they are present in the config
class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceConfigurator(config, prerequisites) {
import ThreadPoolConfigBuilder.conf_?
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig())
val threadPoolConfig: ThreadPoolConfig = createThreadPoolConfigBuilder(config, prerequisites).config
protected def createThreadPoolConfigBuilder(config: Config, prerequisites: DispatcherPrerequisites): ThreadPoolConfigBuilder = {
ThreadPoolConfigBuilder(ThreadPoolConfig())
.setKeepAliveTime(Duration(config getMilliseconds "keep-alive-time", TimeUnit.MILLISECONDS))
.setAllowCoreThreadTimeout(config getBoolean "allow-core-timeout")
.setCorePoolSizeFromFactor(config getInt "core-pool-size-min", config getDouble "core-pool-size-factor", config getInt "core-pool-size-max")
@ -356,4 +375,27 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
case _ None
})(queueFactory _.setQueueFactory(queueFactory)))
}
def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory =
threadPoolConfig.createExecutorServiceFactory(name, threadFactory)
}
class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceConfigurator(config, prerequisites) {
def validate(t: ThreadFactory): ForkJoinPool.ForkJoinWorkerThreadFactory = prerequisites.threadFactory match {
case correct: ForkJoinPool.ForkJoinWorkerThreadFactory correct
case x throw new IllegalStateException("The prerequisites for the ForkJoinExecutorConfigurator is a ForkJoinPool.ForkJoinWorkerThreadFactory!")
}
class ForkJoinExecutorServiceFactory(val threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
val parallelism: Int) extends ExecutorServiceFactory {
def createExecutorService: ExecutorService = new ForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing, true)
}
final def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory =
new ForkJoinExecutorServiceFactory(
validate(threadFactory),
ThreadPoolConfig.scaledPoolSize(
config.getInt("parallelism-min"),
config.getDouble("parallelism-factor"),
config.getInt("parallelism-max")))
}

View file

@ -31,9 +31,9 @@ class BalancingDispatcher(
throughput: Int,
throughputDeadlineTime: Duration,
mailboxType: MailboxType,
config: ThreadPoolConfig,
_executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
_shutdownTimeout: Duration)
extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, config, _shutdownTimeout) {
extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) {
val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
val rebalance = new AtomicBoolean(false)

View file

@ -158,15 +158,14 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
class DispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(config, prerequisites) {
private val instance =
configureThreadPool(config,
threadPoolConfig new Dispatcher(prerequisites,
private val instance = new Dispatcher(
prerequisites,
config.getString("id"),
config.getInt("throughput"),
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType,
threadPoolConfig,
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))).build
configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))
/**
* Returns the same dispatcher instance for each invocation
@ -182,14 +181,13 @@ class DispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisi
class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(config, prerequisites) {
private val instance =
configureThreadPool(config,
threadPoolConfig new BalancingDispatcher(prerequisites,
private val instance = new BalancingDispatcher(
prerequisites,
config.getString("id"),
config.getInt("throughput"),
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
mailboxType, threadPoolConfig,
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))).build
mailboxType, configureExecutor(),
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))
/**
* Returns the same dispatcher instance for each invocation
@ -204,13 +202,23 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP
*/
class PinnedDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(config, prerequisites) {
val threadPoolConfig: ThreadPoolConfig = configureExecutor() match {
case e: ThreadPoolExecutorConfigurator e.threadPoolConfig
case other
prerequisites.eventStream.publish(
Warning("PinnedDispatcherConfigurator",
this.getClass,
"PinnedDispatcher [%s] not configured to use ThreadPoolExecutor, falling back to default config.".format(
config.getString("id"))))
ThreadPoolConfig()
}
/**
* Creates new dispatcher for each invocation.
*/
override def dispatcher(): MessageDispatcher = configureThreadPool(config,
threadPoolConfig
new PinnedDispatcher(prerequisites, null, config.getString("id"), mailboxType,
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS),
threadPoolConfig)).build
override def dispatcher(): MessageDispatcher =
new PinnedDispatcher(
prerequisites, null, config.getString("id"), mailboxType,
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS), threadPoolConfig)
}

View file

@ -340,9 +340,9 @@ object Future {
}
}
sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
sealed trait Future[+T] extends Await.Awaitable[T] {
implicit def executor: ExecutionContext
protected implicit def executor: ExecutionContext
protected final def resolve[X](source: Either[Throwable, X]): Either[Throwable, X] = source match {
case Left(t: scala.runtime.NonLocalReturnControl[_]) Right(t.value.asInstanceOf[X])
@ -362,7 +362,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
case Right(r) that onSuccess { case r2 p success ((r, r2)) }
}
that onFailure { case f p failure f }
p
p.future
}
/**
@ -435,20 +435,20 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
case Left(t) p success t
case Right(r) p failure new NoSuchElementException("Future.failed not completed with a throwable. Instead completed with: " + r)
}
p
p.future
}
/**
* Returns a new Future that will either hold the successful value of this Future,
* or, it this Future fails, it will hold the result of "that" Future.
*/
def or[U >: T](that: Future[U]): Future[U] = {
def fallbackTo[U >: T](that: Future[U]): Future[U] = {
val p = Promise[U]()
onComplete {
case r @ Right(_) p complete r
case _ p completeWith that
}
p
p.future
}
/**
@ -463,12 +463,59 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
* </pre>
*/
final def recover[A >: T](pf: PartialFunction[Throwable, A]): Future[A] = {
val future = Promise[A]()
val p = Promise[A]()
onComplete {
case Left(e) if pf isDefinedAt e future.complete(try { Right(pf(e)) } catch { case x: Exception Left(x) })
case otherwise future complete otherwise
case Left(e) if pf isDefinedAt e p.complete(try { Right(pf(e)) } catch { case x: Exception Left(x) })
case otherwise p complete otherwise
}
future
p.future
}
/**
* Returns a new Future that will, in case this future fails,
* be completed with the resulting Future of the given PartialFunction,
* if the given PartialFunction matches the failure of the original Future.
*
* If the PartialFunction throws, that Throwable will be propagated to the returned Future.
*
* Example:
*
* {{{
* val f = Future { Int.MaxValue }
* Future (6 / 0) recoverWith { case e: ArithmeticException => f } // result: Int.MaxValue
* }}}
*/
def recoverWith[U >: T](pf: PartialFunction[Throwable, Future[U]]): Future[U] = {
val p = Promise[U]()
onComplete {
case Left(t) if pf isDefinedAt t
try { p completeWith pf(t) } catch { case t: Throwable p complete resolve(Left(t)) }
case otherwise p complete otherwise
}
p.future
}
/**
* Returns a new Future that will contain the completed result of this Future,
* and which will invoke the supplied PartialFunction when completed.
*
* This allows for establishing order of side-effects.
*
* {{{
* Future { 5 } andThen {
* case something => assert(something is awesome)
* } andThen {
* case Left(t) => handleProblem(t)
* case Right(v) => dealWithSuccess(v)
* }
* }}}
*/
def andThen[U](pf: PartialFunction[Either[Throwable, T], U]): Future[T] = {
val p = Promise[T]()
onComplete { case r try if (pf isDefinedAt r) pf(r) finally p complete r }
p.future
}
/**
@ -503,6 +550,10 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
/**
* Creates a new Future[A] which is completed with this Future's result if
* that conforms to A's erased type or a ClassCastException otherwise.
*
* When used from Java, to create the Manifest, use:
* import static akka.japi.Util.manifest;
* future.mapTo(manifest(MyClass.class));
*/
final def mapTo[A](implicit m: Manifest[A]): Future[A] = {
val fa = Promise[A]()
@ -515,7 +566,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
case e: ClassCastException Left(e)
})
}
fa
fa.future
}
/**
@ -546,13 +597,13 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
logError("Future.flatMap", e)
}
}
p
p.future
}
/**
* Same as onSuccess { case r => f(r) } but is also used in for-comprehensions
*/
final def foreach(f: T Unit): Unit = onComplete {
final def foreach[U](f: T U): Unit = onComplete {
case Right(r) f(r)
case _
}
@ -586,7 +637,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
Left(e)
})
}
p
p.future
}
protected def logError(msg: String, problem: Throwable): Unit = {
@ -818,3 +869,158 @@ final class KeptPromise[T](suppliedValue: Either[Throwable, T])(implicit val exe
case Right(r) r
}
}
/**
* This class contains bridge classes between Scala and Java.
* Internal use only.
*/
object japi {
@deprecated("Do not use this directly, use subclasses of this", "2.0")
class CallbackBridge[-T] extends PartialFunction[T, Unit] {
override final def isDefinedAt(t: T): Boolean = true
override final def apply(t: T): Unit = internal(t)
protected def internal(result: T): Unit = ()
}
@deprecated("Do not use this directly, use 'Recover'", "2.0")
class RecoverBridge[+T] extends PartialFunction[Throwable, T] {
override final def isDefinedAt(t: Throwable): Boolean = true
override final def apply(t: Throwable): T = internal(t)
protected def internal(result: Throwable): T = null.asInstanceOf[T]
}
@deprecated("Do not use this directly, use subclasses of this", "2.0")
class BooleanFunctionBridge[-T] extends scala.Function1[T, Boolean] {
override final def apply(t: T): Boolean = internal(t)
protected def internal(result: T): Boolean = false
}
@deprecated("Do not use this directly, use subclasses of this", "2.0")
class UnitFunctionBridge[-T] extends (T Unit) {
override final def apply(t: T): Unit = internal(t)
protected def internal(result: T): Unit = ()
}
}
/**
* Callback for when a Future is completed successfully
* SAM (Single Abstract Method) class
*
* Java API
*/
abstract class OnSuccess[-T] extends japi.CallbackBridge[T] {
protected final override def internal(result: T) = onSuccess(result)
/**
* This method will be invoked once when/if a Future that this callback is registered on
* becomes successfully completed
*/
def onSuccess(result: T): Unit
}
/**
* Callback for when a Future is completed with a failure
* SAM (Single Abstract Method) class
*
* Java API
*/
abstract class OnFailure extends japi.CallbackBridge[Throwable] {
protected final override def internal(failure: Throwable) = onFailure(failure)
/**
* This method will be invoked once when/if a Future that this callback is registered on
* becomes completed with a failure
*/
def onFailure(failure: Throwable): Unit
}
/**
* Callback for when a Future is completed with either failure or a success
* SAM (Single Abstract Method) class
*
* Java API
*/
abstract class OnComplete[-T] extends japi.CallbackBridge[Either[Throwable, T]] {
protected final override def internal(value: Either[Throwable, T]): Unit = value match {
case Left(t) onComplete(t, null.asInstanceOf[T])
case Right(r) onComplete(null, r)
}
/**
* This method will be invoked once when/if a Future that this callback is registered on
* becomes completed with a failure or a success.
* In the case of success then "failure" will be null, and in the case of failure the "success" will be null.
*/
def onComplete(failure: Throwable, success: T): Unit
}
/**
* Callback for the Future.recover operation that conditionally turns failures into successes.
*
* SAM (Single Abstract Method) class
*
* Java API
*/
abstract class Recover[+T] extends japi.RecoverBridge[T] {
protected final override def internal(result: Throwable): T = recover(result)
/**
* This method will be invoked once when/if the Future this recover callback is registered on
* becomes completed with a failure.
*
* @returns a successful value for the passed in failure
* @throws the passed in failure to propagate it.
*
* Java API
*/
@throws(classOf[Throwable])
def recover(failure: Throwable): T
}
/**
* Callback for the Future.filter operation that creates a new Future which will
* conditionally contain the success of another Future.
*
* SAM (Single Abstract Method) class
* Java API
*/
abstract class Filter[-T] extends japi.BooleanFunctionBridge[T] {
override final def internal(t: T): Boolean = filter(t)
/**
* This method will be invoked once when/if a Future that this callback is registered on
* becomes completed with a success.
*
* @returns true if the successful value should be propagated to the new Future or not
*/
def filter(result: T): Boolean
}
/**
* Callback for the Future.foreach operation that will be invoked if the Future that this callback
* is registered on becomes completed with a success. This method is essentially the same operation
* as onSuccess.
*
* SAM (Single Abstract Method) class
* Java API
*/
abstract class Foreach[-T] extends japi.UnitFunctionBridge[T] {
override final def internal(t: T): Unit = each(t)
/**
* This method will be invoked once when/if a Future that this callback is registered on
* becomes successfully completed
*/
def each(result: T): Unit
}
/**
* Callback for the Future.map and Future.flatMap operations that will be invoked
* if the Future that this callback is registered on becomes completed with a success.
* This callback is the equivalent of an akka.japi.Function
*
* SAM (Single Abstract Method) class
*
* Java API
*/
abstract class Mapper[-T, +R] extends scala.runtime.AbstractFunction1[T, R]

View file

@ -303,7 +303,7 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
final def enqueue(receiver: ActorRef, handle: Envelope) {
if (pushTimeOut.length > 0) {
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString)
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver)
}
} else queue put handle
}

View file

@ -5,9 +5,20 @@
package akka.dispatch
import java.util.Collection
import java.util.concurrent.atomic.AtomicLong
import akka.util.Duration
import java.util.concurrent._
import akka.jsr166y._
import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.ArrayBlockingQueue
import java.util.concurrent.BlockingQueue
import java.util.concurrent.Callable
import java.util.concurrent.ExecutorService
import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.RejectedExecutionHandler
import java.util.concurrent.RejectedExecutionException
import java.util.concurrent.SynchronousQueue
import java.util.concurrent.TimeUnit
import java.util.concurrent.ThreadFactory
import java.util.concurrent.ThreadPoolExecutor
object ThreadPoolConfig {
type QueueFactory = () BlockingQueue[Runnable]
@ -86,70 +97,65 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def
new ThreadPoolExecutorServiceFactory(threadFactory)
}
trait DispatcherBuilder {
def build: MessageDispatcher
}
object ThreadPoolConfigDispatcherBuilder {
def conf_?[T](opt: Option[T])(fun: (T) ThreadPoolConfigDispatcherBuilder ThreadPoolConfigDispatcherBuilder): Option[(ThreadPoolConfigDispatcherBuilder) ThreadPoolConfigDispatcherBuilder] = opt map fun
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
*/
case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfig) MessageDispatcher, config: ThreadPoolConfig) extends DispatcherBuilder {
case class ThreadPoolConfigBuilder(config: ThreadPoolConfig) {
import ThreadPoolConfig._
def build: MessageDispatcher = dispatcherFactory(config)
def withNewThreadPoolWithCustomBlockingQueue(newQueueFactory: QueueFactory): ThreadPoolConfigDispatcherBuilder =
def withNewThreadPoolWithCustomBlockingQueue(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = newQueueFactory))
def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolConfigDispatcherBuilder =
def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolConfigBuilder =
withNewThreadPoolWithCustomBlockingQueue(reusableQueue(queue))
def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolConfigDispatcherBuilder =
def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = linkedBlockingQueue()))
def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolConfigDispatcherBuilder =
def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = linkedBlockingQueue(capacity)))
def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolConfigDispatcherBuilder =
def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = synchronousQueue(fair)))
def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolConfigDispatcherBuilder =
def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = arrayBlockingQueue(capacity, fair)))
def setCorePoolSize(size: Int): ThreadPoolConfigDispatcherBuilder =
def setCorePoolSize(size: Int): ThreadPoolConfigBuilder =
if (config.maxPoolSize < size)
this.copy(config = config.copy(corePoolSize = size, maxPoolSize = size))
else
this.copy(config = config.copy(corePoolSize = size))
def setMaxPoolSize(size: Int): ThreadPoolConfigDispatcherBuilder =
def setMaxPoolSize(size: Int): ThreadPoolConfigBuilder =
if (config.corePoolSize > size)
this.copy(config = config.copy(corePoolSize = size, maxPoolSize = size))
else
this.copy(config = config.copy(maxPoolSize = size))
def setCorePoolSizeFromFactor(min: Int, multiplier: Double, max: Int): ThreadPoolConfigDispatcherBuilder =
def setCorePoolSizeFromFactor(min: Int, multiplier: Double, max: Int): ThreadPoolConfigBuilder =
setCorePoolSize(scaledPoolSize(min, multiplier, max))
def setMaxPoolSizeFromFactor(min: Int, multiplier: Double, max: Int): ThreadPoolConfigDispatcherBuilder =
def setMaxPoolSizeFromFactor(min: Int, multiplier: Double, max: Int): ThreadPoolConfigBuilder =
setMaxPoolSize(scaledPoolSize(min, multiplier, max))
def setKeepAliveTimeInMillis(time: Long): ThreadPoolConfigDispatcherBuilder =
def setKeepAliveTimeInMillis(time: Long): ThreadPoolConfigBuilder =
setKeepAliveTime(Duration(time, TimeUnit.MILLISECONDS))
def setKeepAliveTime(time: Duration): ThreadPoolConfigDispatcherBuilder =
def setKeepAliveTime(time: Duration): ThreadPoolConfigBuilder =
this.copy(config = config.copy(threadTimeout = time))
def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigDispatcherBuilder =
def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigBuilder =
this.copy(config = config.copy(allowCorePoolTimeout = allow))
def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigDispatcherBuilder =
def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
this.copy(config = config.copy(queueFactory = newQueueFactory))
def configure(fs: Option[Function[ThreadPoolConfigDispatcherBuilder, ThreadPoolConfigDispatcherBuilder]]*): ThreadPoolConfigDispatcherBuilder = fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder = fs.foldLeft(this)((c, f) f.map(_(c)).getOrElse(c))
}
object MonitorableThreadFactory {
@ -161,11 +167,14 @@ case class MonitorableThreadFactory(name: String,
daemonic: Boolean,
contextClassLoader: Option[ClassLoader],
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing)
extends ThreadFactory {
extends ThreadFactory with ForkJoinPool.ForkJoinWorkerThreadFactory {
protected val counter = new AtomicLong
def newThread(runnable: Runnable) = {
val t = new Thread(runnable, name + counter.incrementAndGet())
def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = wire(ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool))
def newThread(runnable: Runnable): Thread = wire(new Thread(runnable, name + counter.incrementAndGet()))
protected def wire[T <: Thread](t: T): T = {
t.setUncaughtExceptionHandler(exceptionHandler)
t.setDaemon(daemonic)
contextClassLoader foreach (t.setContextClassLoader(_))

View file

@ -1,62 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.dispatch.japi
import akka.japi.{ Procedure2, Procedure, Function JFunc }
/* Java API */
trait Future[+T] { self: akka.dispatch.Future[T]
/**
* Asynchronously called when this Future gets a successful result
*/
private[japi] final def onSuccess[A >: T](proc: Procedure[A]): this.type = self.onSuccess({ case r proc(r.asInstanceOf[A]) }: PartialFunction[T, Unit])
/**
* Asynchronously called when this Future gets a failed result
*/
private[japi] final def onFailure(proc: Procedure[Throwable]): this.type = self.onFailure({ case t: Throwable proc(t) }: PartialFunction[Throwable, Unit])
/**
* Asynchronously called when this future is completed with either a failed or a successful result
* In case of a success, the first parameter (Throwable) will be null
* In case of a failure, the second parameter (T) will be null
* For no reason will both be null or neither be null
*/
private[japi] final def onComplete[A >: T](proc: Procedure2[Throwable, A]): this.type = self.onComplete(_.fold(t proc(t, null.asInstanceOf[T]), r proc(null, r)))
/**
* Asynchronously applies the provided function to the (if any) successful result of this Future
* Any failure of this Future will be propagated to the Future returned by this method.
*/
private[japi] final def map[A >: T, B](f: JFunc[A, B]): akka.dispatch.Future[B] = self.map(f(_))
/**
* Asynchronously applies the provided function to the (if any) successful result of this Future and flattens it.
* Any failure of this Future will be propagated to the Future returned by this method.
*/
private[japi] final def flatMap[A >: T, B](f: JFunc[A, akka.dispatch.Future[B]]): akka.dispatch.Future[B] = self.flatMap(f(_))
/**
* Asynchronously applies the provided Procedure to the (if any) successful result of this Future
* Provided Procedure will not be called in case of no-result or in case of failed result
*/
private[japi] final def foreach[A >: T](proc: Procedure[A]): Unit = self.foreach(proc(_))
/**
* Returns a new Future whose successful result will be the successful result of this Future if that result conforms to the provided predicate
* Any failure of this Future will be propagated to the Future returned by this method.
*/
private[japi] final def filter[A >: T](p: JFunc[A, java.lang.Boolean]): akka.dispatch.Future[A] =
self.filter((a: Any) p(a.asInstanceOf[A])).asInstanceOf[akka.dispatch.Future[A]]
/**
* Returns a new Future whose value will be of the specified type if it really is
* Or a failure with a ClassCastException if it wasn't.
*/
private[japi] final def mapTo[A](clazz: Class[A]): akka.dispatch.Future[A] = {
implicit val manifest: Manifest[A] = Manifest.classType(clazz)
self.mapTo[A]
}
}

View file

@ -119,3 +119,13 @@ object Option {
implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala
implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = if (o.isDefined) some(o.get) else none
}
/**
* This class hold common utilities for Java
*/
object Util {
/**
* Given a Class returns a Scala Manifest of that Class
*/
def manifest[T](clazz: Class[T]): Manifest[T] = Manifest.classType(clazz)
}

View file

@ -22,7 +22,69 @@ class AskTimeoutException(message: String, cause: Throwable) extends TimeoutExce
/**
* This object contains implementation details of the ask pattern.
*/
object AskSupport {
trait AskSupport {
/**
* Import this implicit conversion to gain `?` and `ask` methods on
* [[akka.actor.ActorRef]], which will defer to the
* `ask(actorRef, message)(timeout)` method defined here.
*
* {{{
* import akka.pattern.ask
*
* val future = actor ? message // => ask(actor, message)
* val future = actor ask message // => ask(actor, message)
* val future = actor.ask(message)(timeout) // => ask(actor, message)(timeout)
* }}}
*
* All of the above use an implicit [[akka.actor.Timeout]].
*/
implicit def ask(actorRef: ActorRef): AskableActorRef = new AskableActorRef(actorRef)
/**
* Sends a message asynchronously and returns a [[akka.dispatch.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.actor.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>
*
* {{{
* val f = ask(worker, request)(timeout)
* flow {
* EnrichedRequest(request, f())
* } pipeTo nextActor
* }}}
*
* [see [[akka.dispatch.Future]] for a description of `flow`]
*/
def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match {
case ref: InternalActorRef if ref.isTerminated
actorRef.tell(message)
Promise.failed(new AskTimeoutException("sending to terminated ref breaks promises"))(ref.provider.dispatcher)
case ref: InternalActorRef
val provider = ref.provider
if (timeout.duration.length <= 0) {
actorRef.tell(message)
Promise.failed(new AskTimeoutException("not asking with negative timeout"))(provider.dispatcher)
} else {
val a = createAsker(provider, timeout)
actorRef.tell(message, a)
a.result
}
case _ throw new IllegalArgumentException("incompatible ActorRef " + actorRef)
}
/**
* Implementation detail of the ask pattern enrichment of ActorRef
@ -121,7 +183,10 @@ object AskSupport {
}
}
def createAsker(provider: ActorRefProvider, timeout: Timeout): PromiseActorRef = {
/**
* INTERNAL AKKA USE ONLY
*/
private[akka] def createAsker(provider: ActorRefProvider, timeout: Timeout): PromiseActorRef = {
val path = provider.tempPath()
val result = Promise[Any]()(provider.dispatcher)
val a = new PromiseActorRef(provider, path, provider.tempContainer, result, provider.deathWatch)

View file

@ -0,0 +1,47 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.pattern
import akka.actor.{ ActorRef, Actor, ActorSystem, Props, PoisonPill, Terminated, ReceiveTimeout, ActorTimeoutException }
import akka.dispatch.{ Promise, Future }
import akka.util.Duration
trait GracefulStopSupport {
/**
* Returns a [[akka.dispatch.Future]] that will be completed with success (value `true`) when
* existing messages of the target actor has been processed and the actor has been
* terminated.
*
* Useful when you need to wait for termination or compose ordered termination of several actors.
*
* If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]]
* is completed with failure [[akka.actor.ActorTimeoutException]].
*/
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = {
if (target.isTerminated) {
Promise.successful(true)
} else {
val result = Promise[Boolean]()
system.actorOf(Props(new Actor {
// Terminated will be received when target has been stopped
context watch target
target ! PoisonPill
// ReceiveTimeout will be received if nothing else is received within the timeout
context setReceiveTimeout timeout
def receive = {
case Terminated(a) if a == target
result success true
context stop self
case ReceiveTimeout
result failure new ActorTimeoutException(
"Failed to stop [%s] within [%s]".format(target.path, context.receiveTimeout))
context stop self
}
}))
result
}
}
}

View file

@ -6,7 +6,7 @@ package akka.pattern
object Patterns {
import akka.actor.{ ActorRef, ActorSystem }
import akka.dispatch.Future
import akka.pattern.{ ask scalaAsk }
import akka.pattern.{ ask scalaAsk, pipe scalaPipe }
import akka.util.{ Timeout, Duration }
/**
@ -83,10 +83,10 @@ object Patterns {
* // apply some transformation (i.e. enrich with request info)
* final Future<Object> transformed = f.map(new akka.japi.Function<Object, Object>() { ... });
* // send it on to the next stage
* Patterns.pipeTo(transformed, nextActor);
* Patterns.pipe(transformed).to(nextActor);
* }}}
*/
def pipeTo[T](future: Future[T], actorRef: ActorRef): Future[T] = akka.pattern.pipeTo(future, actorRef)
def pipe[T](future: Future[T]): PipeableFuture[T] = scalaPipe(future)
/**
* Returns a [[akka.dispatch.Future]] that will be completed with success (value `true`) when
@ -98,7 +98,6 @@ object Patterns {
* If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]]
* is completed with failure [[akka.actor.ActorTimeoutException]].
*/
def gracefulStop(target: ActorRef, timeout: Duration, system: ActorSystem): Future[java.lang.Boolean] = {
def gracefulStop(target: ActorRef, timeout: Duration, system: ActorSystem): Future[java.lang.Boolean] =
akka.pattern.gracefulStop(target, timeout)(system).asInstanceOf[Future[java.lang.Boolean]]
}
}

View file

@ -3,13 +3,37 @@
*/
package akka.pattern
import akka.actor.ActorRef
import akka.dispatch.Future
import akka.actor.{ Status, ActorRef }
object PipeToSupport {
trait PipeToSupport {
class PipeableFuture[T](val future: Future[T]) {
def pipeTo(actorRef: ActorRef): Future[T] = akka.pattern.pipeTo(future, actorRef)
final class PipeableFuture[T](val future: Future[T]) {
def pipeTo(recipient: ActorRef): Future[T] =
future onComplete {
case Right(r) recipient ! r
case Left(f) recipient ! Status.Failure(f)
}
def to(recipient: ActorRef): PipeableFuture[T] = {
pipeTo(recipient)
this
}
}
/**
* Import this implicit conversion to gain the `pipeTo` method on [[akka.dispatch.Future]]:
*
* {{{
* import akka.pattern.pipe
*
* Future { doExpensiveCalc() } pipeTo nextActor
*
* or
*
* pipe(someFuture) to nextActor
*
* }}}
*/
implicit def pipe[T](future: Future[T]): PipeableFuture[T] = new PipeableFuture(future)
}

View file

@ -40,139 +40,6 @@ import akka.util.{ Timeout, Duration }
* ask(actor, message);
* }}}
*/
package object pattern {
/**
* Import this implicit conversion to gain `?` and `ask` methods on
* [[akka.actor.ActorRef]], which will defer to the
* `ask(actorRef, message)(timeout)` method defined here.
*
* {{{
* import akka.pattern.ask
*
* val future = actor ? message // => ask(actor, message)
* val future = actor ask message // => ask(actor, message)
* val future = actor.ask(message)(timeout) // => ask(actor, message)(timeout)
* }}}
*
* All of the above use an implicit [[akka.actor.Timeout]].
*/
implicit def ask(actorRef: ActorRef): AskSupport.AskableActorRef = new AskSupport.AskableActorRef(actorRef)
/**
* Sends a message asynchronously and returns a [[akka.dispatch.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.actor.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>
*
* {{{
* val f = ask(worker, request)(timeout)
* flow {
* EnrichedRequest(request, f())
* } pipeTo nextActor
* }}}
*
* [see [[akka.dispatch.Future]] for a description of `flow`]
*/
def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match {
case ref: InternalActorRef if ref.isTerminated
actorRef.tell(message)
Promise.failed(new AskTimeoutException("sending to terminated ref breaks promises"))(ref.provider.dispatcher)
case ref: InternalActorRef
val provider = ref.provider
if (timeout.duration.length <= 0) {
actorRef.tell(message)
Promise.failed(new AskTimeoutException("not asking with negative timeout"))(provider.dispatcher)
} else {
val a = AskSupport.createAsker(provider, timeout)
actorRef.tell(message, a)
a.result
}
case _ throw new IllegalArgumentException("incompatible ActorRef " + actorRef)
}
/**
* Import this implicit conversion to gain the `pipeTo` method on [[akka.dispatch.Future]]:
*
* {{{
* import akka.pattern.pipeTo
*
* Future { doExpensiveCalc() } pipeTo nextActor
* }}}
*/
implicit def pipeTo[T](future: Future[T]): PipeToSupport.PipeableFuture[T] = new PipeToSupport.PipeableFuture(future)
/**
* Register an onComplete callback on this [[akka.dispatch.Future]] to send
* the result to the given actor reference. Returns the original Future to
* allow method chaining.
*
* <b>Recommended usage example:</b>
*
* {{{
* val f = ask(worker, request)(timeout)
* flow {
* EnrichedRequest(request, f())
* } pipeTo nextActor
* }}}
*
* [see [[akka.dispatch.Future]] for a description of `flow`]
*/
def pipeTo[T](future: Future[T], actorRef: ActorRef): Future[T] = {
future onComplete {
case Right(r) actorRef ! r
case Left(f) actorRef ! Status.Failure(f)
}
future
}
/**
* Returns a [[akka.dispatch.Future]] that will be completed with success (value `true`) when
* existing messages of the target actor has been processed and the actor has been
* terminated.
*
* Useful when you need to wait for termination or compose ordered termination of several actors.
*
* If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]]
* is completed with failure [[akka.actor.ActorTimeoutException]].
*/
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = {
if (target.isTerminated) {
Promise.successful(true)
} else {
val result = Promise[Boolean]()
system.actorOf(Props(new Actor {
// Terminated will be received when target has been stopped
context watch target
target ! PoisonPill
// ReceiveTimeout will be received if nothing else is received within the timeout
context setReceiveTimeout timeout
def receive = {
case Terminated(a) if a == target
result success true
context stop self
case ReceiveTimeout
result failure new ActorTimeoutException(
"Failed to stop [%s] within [%s]".format(target.path, context.receiveTimeout))
context stop self
}
}))
result
}
}
package object pattern extends PipeToSupport with AskSupport with GracefulStopSupport {
}

View file

@ -10,7 +10,7 @@ import akka.util.Duration
import akka.util.duration._
import com.typesafe.config.Config
import akka.config.ConfigurationException
import akka.pattern.AskSupport
import akka.pattern.pipe
import scala.collection.JavaConversions.iterableAsScalaIterable
/**
@ -766,7 +766,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒
{
case (sender, message)
val provider: ActorRefProvider = routeeProvider.context.asInstanceOf[ActorCell].systemImpl.provider
val asker = AskSupport.createAsker(provider, within)
val asker = akka.pattern.createAsker(provider, within)
asker.result.pipeTo(sender)
toAll(asker, routeeProvider.routees)
}

View file

@ -3,9 +3,8 @@
*/
package akka.util
import java.{ lang jl }
object BoxedType {
import java.{ lang jl }
private val toBoxed = Map[Class[_], Class[_]](
classOf[Boolean] -> classOf[jl.Boolean],
@ -18,8 +17,5 @@ object BoxedType {
classOf[Double] -> classOf[jl.Double],
classOf[Unit] -> classOf[scala.runtime.BoxedUnit])
def apply(c: Class[_]): Class[_] = {
if (c.isPrimitive) toBoxed(c) else c
}
final def apply(c: Class[_]): Class[_] = if (c.isPrimitive) toBoxed(c) else c
}

View file

@ -8,40 +8,13 @@ import java.util.concurrent.TimeUnit
import TimeUnit._
import java.lang.{ Double JDouble }
class TimerException(message: String) extends RuntimeException(message)
/**
* Simple timer class.
* Usage:
* <pre>
* import akka.util.duration._
* import akka.util.Timer
*
* val timer = Timer(30.seconds)
* while (timer.isTicking) { ... }
* </pre>
*/
case class Timer(duration: Duration, throwExceptionOnTimeout: Boolean = false) {
val startTimeInMillis = System.currentTimeMillis
val timeoutInMillis = duration.toMillis
/**
* Returns true while the timer is ticking. After that it either throws and exception or
* returns false. Depending on if the 'throwExceptionOnTimeout' argument is true or false.
*/
def isTicking: Boolean = {
if (!(timeoutInMillis > (System.currentTimeMillis - startTimeInMillis))) {
if (throwExceptionOnTimeout) throw new TimerException("Time out after " + duration)
else false
} else true
}
}
case class Deadline(d: Duration) {
def +(other: Duration): Deadline = copy(d = d + other)
def -(other: Duration): Deadline = copy(d = d - other)
def -(other: Deadline): Duration = d - other.d
case class Deadline private (time: Duration) {
def +(other: Duration): Deadline = copy(time = time + other)
def -(other: Duration): Deadline = copy(time = time - other)
def -(other: Deadline): Duration = time - other.time
def timeLeft: Duration = this - Deadline.now
def hasTimeLeft(): Boolean = !isOverdue() //Code reuse FTW
def isOverdue(): Boolean = (time.toNanos - System.nanoTime()) < 0
}
object Deadline {
def now: Deadline = Deadline(Duration(System.nanoTime, NANOSECONDS))

View file

@ -1,34 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
import akka.event.Logging.Error
import java.lang.management.ManagementFactory
import javax.management.{ ObjectInstance, ObjectName, InstanceAlreadyExistsException, InstanceNotFoundException }
import akka.actor.ActorSystem
object JMX {
private val mbeanServer = ManagementFactory.getPlatformMBeanServer
def nameFor(hostname: String, service: String, bean: String): ObjectName =
new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_")))
def register(name: ObjectName, mbean: AnyRef)(implicit system: ActorSystem): Option[ObjectInstance] = try {
Some(mbeanServer.registerMBean(mbean, name))
} catch {
case e: InstanceAlreadyExistsException
Some(mbeanServer.getObjectInstance(name))
case e: Exception
system.eventStream.publish(Error(e, "JMX", this.getClass, "Error when registering mbean [%s]".format(mbean)))
None
}
def unregister(mbean: ObjectName)(implicit system: ActorSystem) = try {
mbeanServer.unregisterMBean(mbean)
} catch {
case e: InstanceNotFoundException {}
case e: Exception system.eventStream.publish(Error(e, "JMX", this.getClass, "Error while unregistering mbean [%s]".format(mbean)))
}
}

View file

@ -1,187 +0,0 @@
package akka.cluster;
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import org.apache.bookkeeper.proto.BookieServer;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.server.NIOServerCnxnFactory;
import org.apache.zookeeper.server.ZooKeeperServer;
public class LocalBookKeeper {
public static final int CONNECTION_TIMEOUT = 30000;
int numberOfBookies;
public LocalBookKeeper() {
numberOfBookies = 3;
}
public LocalBookKeeper(int numberOfBookies) {
this();
this.numberOfBookies = numberOfBookies;
}
private final String HOSTPORT = "127.0.0.1:2181";
NIOServerCnxnFactory serverFactory;
ZooKeeperServer zks;
ZooKeeper zkc;
int ZooKeeperDefaultPort = 2181;
File ZkTmpDir;
//BookKeeper variables
File tmpDirs[];
BookieServer bs[];
Integer initialPort = 5000;
/**
* @param args
*/
public void runZookeeper(int maxCC) throws IOException{
// create a ZooKeeper server(dataDir, dataLogDir, port)
//ServerStats.registerAsConcrete();
//ClientBase.setupTestEnv();
ZkTmpDir = File.createTempFile("zookeeper", "test");
ZkTmpDir.delete();
ZkTmpDir.mkdir();
try {
zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
serverFactory = new NIOServerCnxnFactory();
serverFactory.configure(new InetSocketAddress(ZooKeeperDefaultPort), maxCC);
serverFactory.startup(zks);
} catch (Exception e) {
// TODO Auto-generated catch block
}
boolean b = waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT);
}
public void initializeZookeper() {
//initialize the zk client with values
try {
zkc = new ZooKeeper("127.0.0.1", ZooKeeperDefaultPort, new emptyWatcher());
zkc.create("/ledgers", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
zkc.create("/ledgers/available", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
// No need to create an entry for each requested bookie anymore as the
// BookieServers will register themselves with ZooKeeper on startup.
} catch (KeeperException e) {
} catch (InterruptedException e) {
} catch (IOException e) {
}
}
public void runBookies() throws IOException{
// Create Bookie Servers (B1, B2, B3)
tmpDirs = new File[numberOfBookies];
bs = new BookieServer[numberOfBookies];
for(int i = 0; i < numberOfBookies; i++) {
tmpDirs[i] = File.createTempFile("bookie" + Integer.toString(i), "test");
tmpDirs[i].delete();
tmpDirs[i].mkdir();
bs[i] = new BookieServer(initialPort + i, InetAddress.getLocalHost().getHostAddress() + ":"
+ ZooKeeperDefaultPort, tmpDirs[i], new File[]{tmpDirs[i]});
bs[i].start();
}
}
public static void main(String[] args) throws IOException, InterruptedException {
if(args.length < 1) {
usage();
System.exit(-1);
}
LocalBookKeeper lb = new LocalBookKeeper(Integer.parseInt(args[0]));
lb.runZookeeper(1000);
lb.initializeZookeper();
lb.runBookies();
while (true) {
Thread.sleep(5000);
}
}
private static void usage() {
System.err.println("Usage: LocalBookKeeper number-of-bookies");
}
/* User for testing purposes, void */
class emptyWatcher implements Watcher{
public void process(WatchedEvent event) {}
}
public static boolean waitForServerUp(String hp, long timeout) {
long start = System.currentTimeMillis();
String split[] = hp.split(":");
String host = split[0];
int port = Integer.parseInt(split[1]);
while (true) {
try {
Socket sock = new Socket(host, port);
BufferedReader reader = null;
try {
OutputStream outstream = sock.getOutputStream();
outstream.write("stat".getBytes());
outstream.flush();
reader =
new BufferedReader(
new InputStreamReader(sock.getInputStream()));
String line = reader.readLine();
if (line != null && line.startsWith("Zookeeper version:")) {
return true;
}
} finally {
sock.close();
if (reader != null) {
reader.close();
}
}
} catch (IOException e) {
// ignore as this is expected
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
}
return false;
}
}

View file

@ -1,312 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package akka.cluster.zookeeper;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.TreeMap;
import java.util.concurrent.CountDownLatch;
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
/**
*
* A <a href="package.html">protocol to implement a distributed queue</a>.
*
*/
public class DistributedQueue {
private static final Logger LOG = Logger.getLogger(DistributedQueue.class);
private final String dir;
private ZooKeeper zookeeper;
private List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
private final String prefix = "qn-";
public DistributedQueue(ZooKeeper zookeeper, String dir, List<ACL> acl) {
this.dir = dir;
if(acl != null) {
this.acl = acl;
}
this.zookeeper = zookeeper;
}
/**
* Returns a Map of the children, ordered by id.
* @param watcher optional watcher on getChildren() operation.
* @return map from id to child name for all children
*/
private TreeMap<Long,String> orderedChildren(Watcher watcher) throws KeeperException, InterruptedException {
TreeMap<Long,String> orderedChildren = new TreeMap<Long,String>();
List<String> childNames = null;
try{
childNames = zookeeper.getChildren(dir, watcher);
}catch (KeeperException.NoNodeException e) {
throw e;
}
for(String childName : childNames) {
try{
//Check format
if(!childName.regionMatches(0, prefix, 0, prefix.length())) {
LOG.warn("Found child node with improper name: " + childName);
continue;
}
String suffix = childName.substring(prefix.length());
Long childId = new Long(suffix);
orderedChildren.put(childId,childName);
}catch(NumberFormatException e) {
LOG.warn("Found child node with improper format : " + childName + " " + e,e);
}
}
return orderedChildren;
}
/**
* Find the smallest child node.
* @return The name of the smallest child node.
*/
private String smallestChildName() throws KeeperException, InterruptedException {
long minId = Long.MAX_VALUE;
String minName = "";
List<String> childNames = null;
try{
childNames = zookeeper.getChildren(dir, false);
}catch(KeeperException.NoNodeException e) {
LOG.warn("Caught: " +e,e);
return null;
}
for(String childName : childNames) {
try{
//Check format
if(!childName.regionMatches(0, prefix, 0, prefix.length())) {
LOG.warn("Found child node with improper name: " + childName);
continue;
}
String suffix = childName.substring(prefix.length());
long childId = Long.parseLong(suffix);
if(childId < minId) {
minId = childId;
minName = childName;
}
}catch(NumberFormatException e) {
LOG.warn("Found child node with improper format : " + childName + " " + e,e);
}
}
if(minId < Long.MAX_VALUE) {
return minName;
}else{
return null;
}
}
/**
* Return the head of the queue without modifying the queue.
* @return the data at the head of the queue.
* @throws NoSuchElementException
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] element() throws NoSuchElementException, KeeperException, InterruptedException {
TreeMap<Long,String> orderedChildren;
// element, take, and remove follow the same pattern.
// We want to return the child node with the smallest sequence number.
// Since other clients are remove()ing and take()ing nodes concurrently,
// the child with the smallest sequence number in orderedChildren might be gone by the time we check.
// We don't call getChildren again until we have tried the rest of the nodes in sequence order.
while(true) {
try{
orderedChildren = orderedChildren(null);
}catch(KeeperException.NoNodeException e) {
throw new NoSuchElementException();
}
if(orderedChildren.size() == 0 ) throw new NoSuchElementException();
for(String headNode : orderedChildren.values()) {
if(headNode != null) {
try{
return zookeeper.getData(dir+"/"+headNode, false, null);
}catch(KeeperException.NoNodeException e) {
//Another client removed the node first, try next
}
}
}
}
}
/**
* Attempts to remove the head of the queue and return it.
* @return The former head of the queue
* @throws NoSuchElementException
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException {
TreeMap<Long,String> orderedChildren;
// Same as for element. Should refactor this.
while(true) {
try{
orderedChildren = orderedChildren(null);
}catch(KeeperException.NoNodeException e) {
throw new NoSuchElementException();
}
if(orderedChildren.size() == 0) throw new NoSuchElementException();
for(String headNode : orderedChildren.values()) {
String path = dir +"/"+headNode;
try{
byte[] data = zookeeper.getData(path, false, null);
zookeeper.delete(path, -1);
return data;
}catch(KeeperException.NoNodeException e) {
// Another client deleted the node first.
}
}
}
}
private class LatchChildWatcher implements Watcher {
CountDownLatch latch;
public LatchChildWatcher() {
latch = new CountDownLatch(1);
}
public void process(WatchedEvent event) {
LOG.debug("Watcher fired on path: " + event.getPath() + " state: " +
event.getState() + " type " + event.getType());
latch.countDown();
}
public void await() throws InterruptedException {
latch.await();
}
}
/**
* Removes the head of the queue and returns it, blocks until it succeeds.
* @return The former head of the queue
* @throws NoSuchElementException
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] take() throws KeeperException, InterruptedException {
TreeMap<Long,String> orderedChildren;
// Same as for element. Should refactor this.
while(true) {
LatchChildWatcher childWatcher = new LatchChildWatcher();
try{
orderedChildren = orderedChildren(childWatcher);
}catch(KeeperException.NoNodeException e) {
zookeeper.create(dir, new byte[0], acl, CreateMode.PERSISTENT);
continue;
}
if(orderedChildren.size() == 0) {
childWatcher.await();
continue;
}
for(String headNode : orderedChildren.values()) {
String path = dir +"/"+headNode;
try{
byte[] data = zookeeper.getData(path, false, null);
zookeeper.delete(path, -1);
return data;
}catch(KeeperException.NoNodeException e) {
// Another client deleted the node first.
}
}
}
}
/**
* Inserts data into queue.
* @param data
* @return true if data was successfully added
*/
public boolean offer(byte[] data) throws KeeperException, InterruptedException{
for(;;) {
try{
zookeeper.create(dir+"/"+prefix, data, acl, CreateMode.PERSISTENT_SEQUENTIAL);
return true;
}catch(KeeperException.NoNodeException e) {
zookeeper.create(dir, new byte[0], acl, CreateMode.PERSISTENT);
}
}
}
/**
* Returns the data at the first element of the queue, or null if the queue is empty.
* @return data at the first element of the queue, or null.
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] peek() throws KeeperException, InterruptedException{
try{
return element();
}catch(NoSuchElementException e) {
return null;
}
}
/**
* Attempts to remove the head of the queue and return it. Returns null if the queue is empty.
* @return Head of the queue or null.
* @throws KeeperException
* @throws InterruptedException
*/
public byte[] poll() throws KeeperException, InterruptedException {
try{
return remove();
}catch(NoSuchElementException e) {
return null;
}
}
}

View file

@ -1,173 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.zookeeper;
import java.io.Serializable;
import java.util.List;
import java.util.ArrayList;
import org.I0Itec.zkclient.ExceptionUtil;
import org.I0Itec.zkclient.IZkChildListener;
import org.I0Itec.zkclient.ZkClient;
import org.I0Itec.zkclient.exception.ZkNoNodeException;
public class ZooKeeperQueue<T extends Object> {
protected static class Element<T> {
private String _name;
private T _data;
public Element(String name, T data) {
_name = name;
_data = data;
}
public String getName() {
return _name;
}
public T getData() {
return _data;
}
}
protected final ZkClient _zkClient;
private final String _elementsPath;
private final String _rootPath;
private final boolean _isBlocking;
public ZooKeeperQueue(ZkClient zkClient, String rootPath, boolean isBlocking) {
_zkClient = zkClient;
_rootPath = rootPath;
_isBlocking = isBlocking;
_elementsPath = rootPath + "/queue";
if (!_zkClient.exists(rootPath)) {
_zkClient.createPersistent(rootPath, true);
_zkClient.createPersistent(_elementsPath, true);
}
}
public String enqueue(T element) {
try {
String sequential = _zkClient.createPersistentSequential(getElementRoughPath(), element);
String elementId = sequential.substring(sequential.lastIndexOf('/') + 1);
return elementId;
} catch (Exception e) {
throw ExceptionUtil.convertToRuntimeException(e);
}
}
public T dequeue() throws InterruptedException {
if (_isBlocking) {
Element<T> element = getFirstElement();
_zkClient.delete(getElementPath(element.getName()));
return element.getData();
} else {
throw new UnsupportedOperationException("Non-blocking ZooKeeperQueue is not yet supported");
/* FIXME DOES NOT WORK
try {
String headName = getSmallestElement(_zkClient.getChildren(_elementsPath));
String headPath = getElementPath(headName);
return (T) _zkClient.readData(headPath);
} catch (ZkNoNodeException e) {
return null;
}
*/
}
}
public boolean containsElement(String elementId) {
String zkPath = getElementPath(elementId);
return _zkClient.exists(zkPath);
}
public T peek() throws InterruptedException {
Element<T> element = getFirstElement();
if (element == null) {
return null;
}
return element.getData();
}
@SuppressWarnings("unchecked")
public List<T> getElements() {
List<String> paths =_zkClient.getChildren(_elementsPath);
List<T> elements = new ArrayList<T>();
for (String path: paths) {
elements.add((T)_zkClient.readData(path));
}
return elements;
}
public int size() {
return _zkClient.getChildren(_elementsPath).size();
}
public void clear() {
_zkClient.deleteRecursive(_rootPath);
}
public boolean isEmpty() {
return size() == 0;
}
private String getElementRoughPath() {
return getElementPath("item" + "-");
}
private String getElementPath(String elementId) {
return _elementsPath + "/" + elementId;
}
private String getSmallestElement(List<String> list) {
String smallestElement = list.get(0);
for (String element : list) {
if (element.compareTo(smallestElement) < 0) {
smallestElement = element;
}
}
return smallestElement;
}
@SuppressWarnings("unchecked")
protected Element<T> getFirstElement() throws InterruptedException {
final Object mutex = new Object();
IZkChildListener notifyListener = new IZkChildListener() {
@Override
public void handleChildChange(String parentPath, List<String> currentChilds) throws Exception {
synchronized (mutex) {
mutex.notify();
}
}
};
try {
while (true) {
List<String> elementNames;
synchronized (mutex) {
elementNames = _zkClient.subscribeChildChanges(_elementsPath, notifyListener);
while (elementNames == null || elementNames.isEmpty()) {
mutex.wait();
elementNames = _zkClient.getChildren(_elementsPath);
}
}
String elementName = getSmallestElement(elementNames);
try {
String elementPath = getElementPath(elementName);
return new Element<T>(elementName, (T) _zkClient.readData(elementPath));
} catch (ZkNoNodeException e) {
// somebody else picked up the element first, so we have to
// retry with the new first element
}
}
} catch (InterruptedException e) {
throw e;
} catch (Exception e) {
throw ExceptionUtil.convertToRuntimeException(e);
} finally {
_zkClient.unsubscribeChildChanges(_elementsPath, notifyListener);
}
}
}

View file

@ -0,0 +1,33 @@
######################################
# Akka Cluster Reference Config File #
######################################
# This the reference config file has all the default settings.
# Make your edits/overrides in your application.conf.
akka {
cluster {
seed-nodes = []
seed-node-connection-timeout = 30s
max-time-to-retry-joining-cluster = 30s
# accrual failure detection config
failure-detector {
# defines the failure detector threshold
# A low threshold is prone to generate many wrong suspicions but ensures
# a quick detection in the event of a real crash. Conversely, a high
# threshold generates fewer mistakes but needs more time to detect
# actual crashes
threshold = 8
max-sample-size = 1000
}
gossip {
initialDelay = 5s
frequency = 1s
}
}
}

View file

@ -2,13 +2,16 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
package akka.cluster
import akka.actor.{ ActorSystem, Address }
import akka.event.Logging
import java.util.concurrent.atomic.AtomicReference
import scala.collection.immutable.Map
import scala.annotation.tailrec
import java.util.concurrent.atomic.AtomicReference
import System.{ currentTimeMillis newTimestamp }
import akka.actor.{ ActorSystem, Address }
/**
* Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper:
@ -20,12 +23,14 @@ import akka.actor.{ ActorSystem, Address }
* <p/>
* Default threshold is 8, but can be configured in the Akka config.
*/
class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 1000) {
class AccrualFailureDetector(system: ActorSystem, val threshold: Int = 8, val maxSampleSize: Int = 1000) {
private final val PhiFactor = 1.0 / math.log(10.0)
private case class FailureStats(mean: Double = 0.0D, variance: Double = 0.0D, deviation: Double = 0.0D)
private val log = Logging(system, "FailureDetector")
/**
* Implement using optimistic lockless concurrency, all state is represented
* by this immutable case class and managed by an AtomicReference.
@ -49,6 +54,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
*/
@tailrec
final def heartbeat(connection: Address) {
log.debug("Heartbeat from connection [{}] ", connection)
val oldState = state.get
val latestTimestamp = oldState.timestamps.get(connection)
@ -132,12 +138,15 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
def phi(connection: Address): Double = {
val oldState = state.get
val oldTimestamp = oldState.timestamps.get(connection)
val phi =
if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
else {
val timestampDiff = newTimestamp - oldTimestamp.get
val mean = oldState.failureStats.get(connection).getOrElse(FailureStats()).mean
PhiFactor * timestampDiff / mean
}
log.debug("Phi value [{}] and threshold [{}] for connection [{}] ", phi, threshold, connection)
phi
}
/**

View file

@ -1,35 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import org.apache.bookkeeper.proto.BookieServer
import java.io.File
/*
A simple use of BookKeeper is to implement a write-ahead transaction log. A server maintains an in-memory data structure
(with periodic snapshots for example) and logs changes to that structure before it applies the change. The system
server creates a ledger at startup and store the ledger id and password in a well known place (ZooKeeper maybe). When
it needs to make a change, the server adds an entry with the change information to a ledger and apply the change when
BookKeeper adds the entry successfully. The server can even use asyncAddEntry to queue up many changes for high change
throughput. BooKeeper meticulously logs the changes in order and call the completion functions in order.
When the system server dies, a backup server will come online, get the last snapshot and then it will open the
ledger of the old server and read all the entries from the time the snapshot was taken. (Since it doesn't know the last
entry number it will use MAX_INTEGER). Once all the entries have been processed, it will close the ledger and start a
new one for its use.
*/
object BookKeeperServer {
val port = 3181
val zkServers = "localhost:2181"
val journal = new File("./bk/journal")
val ledgers = Array(new File("./bk/ledger"))
val bookie = new BookieServer(port, zkServers, journal, ledgers)
def start() {
bookie.start()
bookie.join()
}
}

File diff suppressed because it is too large Load diff

View file

@ -1,129 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor._
import akka.util._
import ReflectiveAccess._
import akka.routing._
import akka.cluster._
import FailureDetector._
import akka.event.EventHandler
import akka.config.ConfigurationException
import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicReference
import collection.immutable.Map
import annotation.tailrec
/**
* ClusterActorRef factory and locator.
*/
object ClusterActorRef {
import FailureDetectorType._
import RouterType._
def newRef(
actorAddress: String,
routerType: RouterType,
failureDetectorType: FailureDetectorType,
timeout: Long): ClusterActorRef = {
val routerFactory: () Router = routerType match {
case Direct () new DirectRouter
case Random () new RandomRouter
case RoundRobin () new RoundRobinRouter
case LeastCPU sys.error("Router LeastCPU not supported yet")
case LeastRAM sys.error("Router LeastRAM not supported yet")
case LeastMessages sys.error("Router LeastMessages not supported yet")
case Custom sys.error("Router Custom not supported yet")
}
val failureDetectorFactory: (Map[InetSocketAddress, ActorRef]) FailureDetector = failureDetectorType match {
case RemoveConnectionOnFirstFailureLocalFailureDetector
(connections: Map[InetSocketAddress, ActorRef]) new RemoveConnectionOnFirstFailureLocalFailureDetector(connections.values)
case RemoveConnectionOnFirstFailureRemoteFailureDetector
(connections: Map[InetSocketAddress, ActorRef]) new RemoveConnectionOnFirstFailureRemoteFailureDetector(connections)
case CustomFailureDetector(implClass)
(connections: Map[InetSocketAddress, ActorRef]) FailureDetector.createCustomFailureDetector(implClass, connections)
}
new ClusterActorRef(
RoutedProps()
.withTimeout(timeout)
.withRouter(routerFactory)
.withFailureDetector(failureDetectorFactory),
actorAddress)
}
/**
* Finds the cluster actor reference that has a specific address.
*/
def actorFor(address: String): Option[ActorRef] =
Actor.registry.local.actorFor(Address.clusterActorRefPrefix + address)
private[cluster] def createRemoteActorRef(actorAddress: String, inetSocketAddress: InetSocketAddress) = {
RemoteActorRef(inetSocketAddress, actorAddress, Actor.TIMEOUT, None)
}
}
/**
* ActorRef representing a one or many instances of a clustered, load-balanced and sometimes replicated actor
* where the instances can reside on other nodes in the cluster.
*/
private[akka] class ClusterActorRef(props: RoutedProps, val address: String) extends AbstractRoutedActorRef(props) {
import ClusterActorRef._
ClusterModule.ensureEnabled()
val addresses = Cluster.node.inetSocketAddressesForActor(address)
EventHandler.debug(this,
"Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]"
.format(address, router, Cluster.node.remoteServerAddress, addresses.map(_._2).mkString("\n\t")))
addresses foreach {
case (_, address) Cluster.node.clusterActorRefs.put(address, this)
}
val connections: FailureDetector = {
val remoteConnections = (Map[InetSocketAddress, ActorRef]() /: addresses) {
case (map, (uuid, inetSocketAddress))
map + (inetSocketAddress -> createRemoteActorRef(address, inetSocketAddress))
}
props.failureDetectorFactory(remoteConnections)
}
router.init(connections)
def nrOfConnections: Int = connections.size
private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress) {
connections.failOver(from, to)
}
def stop() {
synchronized {
if (_status == ActorRefInternals.RUNNING) {
Actor.registry.local.unregisterClusterActorRef(this)
_status = ActorRefInternals.SHUTDOWN
postMessageToMailbox(Terminate, None)
// FIXME here we need to fire off Actor.cluster.remove(address) (which needs to be properly implemented first, see ticket)
connections.stopAll()
}
}
}
/* If you start me up */
if (_status == ActorRefInternals.UNSTARTED) {
_status = ActorRefInternals.RUNNING
Actor.registry.local.registerClusterActorRef(this)
}
}

View file

@ -1,205 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor.DeploymentConfig._
import akka.actor._
import akka.event.EventHandler
import akka.config.Config
import akka.util.Switch
import akka.util.Helpers._
import akka.cluster.zookeeper.AkkaZkClient
import org.apache.zookeeper.CreateMode
import org.apache.zookeeper.recipes.lock.{ WriteLock, LockListener }
import org.I0Itec.zkclient.exception.{ ZkNoNodeException, ZkNodeExistsException }
import scala.collection.immutable.Seq
import scala.collection.JavaConversions.collectionAsScalaIterable
import java.util.concurrent.{ CountDownLatch, TimeUnit }
/**
* A ClusterDeployer is responsible for deploying a Deploy.
*/
object ClusterDeployer extends ActorDeployer {
val clusterName = Cluster.name
val nodeName = Config.nodename
val clusterPath = "/%s" format clusterName
val deploymentPath = clusterPath + "/deployment"
val deploymentAddressPath = deploymentPath + "/%s"
val deploymentCoordinationPath = clusterPath + "/deployment-coordination"
val deploymentInProgressLockPath = deploymentCoordinationPath + "/in-progress"
val isDeploymentCompletedInClusterLockPath = deploymentCoordinationPath + "/completed" // should not be part of basePaths
val basePaths = List(clusterPath, deploymentPath, deploymentCoordinationPath, deploymentInProgressLockPath)
private val isConnected = new Switch(false)
private val deploymentCompleted = new CountDownLatch(1)
private val zkClient = new AkkaZkClient(
Cluster.zooKeeperServers,
Cluster.sessionTimeout,
Cluster.connectionTimeout,
Cluster.defaultZooKeeperSerializer)
private val deploymentInProgressLockListener = new LockListener {
def lockAcquired() {
EventHandler.info(this, "Clustered deployment started")
}
def lockReleased() {
EventHandler.info(this, "Clustered deployment completed")
deploymentCompleted.countDown()
}
}
private val deploymentInProgressLock = new WriteLock(
zkClient.connection.getZookeeper,
deploymentInProgressLockPath,
null,
deploymentInProgressLockListener)
private val systemDeployments: List[Deploy] = Nil
def shutdown() {
isConnected switchOff {
// undeploy all
try {
for {
child collectionAsScalaIterable(zkClient.getChildren(deploymentPath))
deployment zkClient.readData(deploymentAddressPath.format(child)).asInstanceOf[Deploy]
} zkClient.delete(deploymentAddressPath.format(deployment.address))
invalidateDeploymentInCluster()
} catch {
case e: Exception
handleError(new DeploymentException("Could not undeploy all deployment data in ZooKeeper due to: " + e))
}
// shut down ZooKeeper client
zkClient.close()
EventHandler.info(this, "ClusterDeployer shut down successfully")
}
}
def lookupDeploymentFor(address: String): Option[Deploy] = ensureRunning {
LocalDeployer.lookupDeploymentFor(address) match { // try local cache
case Some(deployment) // in local cache
deployment
case None // not in cache, check cluster
val deployment =
try {
Some(zkClient.readData(deploymentAddressPath.format(address)).asInstanceOf[Deploy])
} catch {
case e: ZkNoNodeException None
case e: Exception
EventHandler.warning(this, e.toString)
None
}
deployment foreach (LocalDeployer.deploy(_)) // cache it in local cache
deployment
}
}
def fetchDeploymentsFromCluster: List[Deploy] = ensureRunning {
val addresses =
try {
zkClient.getChildren(deploymentPath).toList
} catch {
case e: ZkNoNodeException List[String]()
}
val deployments = addresses map { address
zkClient.readData(deploymentAddressPath.format(address)).asInstanceOf[Deploy]
}
EventHandler.info(this, "Fetched deployment plans from cluster [\n\t%s\n]" format deployments.mkString("\n\t"))
deployments
}
private[akka] def init(deployments: Seq[Deploy]) {
isConnected switchOn {
EventHandler.info(this, "Initializing ClusterDeployer")
basePaths foreach { path
try {
ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT))
EventHandler.debug(this, "Created ZooKeeper path for deployment [%s]".format(path))
} catch {
case e
val error = new DeploymentException(e.toString)
EventHandler.error(error, this)
throw error
}
}
val allDeployments = deployments ++ systemDeployments
if (!isDeploymentCompletedInCluster) {
if (deploymentInProgressLock.lock()) {
// try to be the one doing the clustered deployment
EventHandler.info(this, "Pushing clustered deployment plans [\n\t" + allDeployments.mkString("\n\t") + "\n]")
allDeployments foreach (deploy(_)) // deploy
markDeploymentCompletedInCluster()
deploymentInProgressLock.unlock() // signal deployment complete
} else {
deploymentCompleted.await(30, TimeUnit.SECONDS) // wait until deployment is completed by other "master" node
}
}
// fetch clustered deployments and deploy them locally
fetchDeploymentsFromCluster foreach (LocalDeployer.deploy(_))
}
}
private[akka] def deploy(deployment: Deploy) {
ensureRunning {
LocalDeployer.deploy(deployment)
deployment match {
case Deploy(_, _, _, _, Local) | Deploy(_, _, _, _, _: Local) //TODO LocalDeployer.deploy(deployment)??
case Deploy(address, recipe, routing, _, _) // cluster deployment
/*TODO recipe foreach { r ⇒
Deployer.newClusterActorRef(() Actor.actorOf(r.implementationClass), address, deployment)
}*/
val path = deploymentAddressPath.format(address)
try {
ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT))
zkClient.writeData(path, deployment)
} catch {
case e: NullPointerException
handleError(new DeploymentException(
"Could not store deployment data [" + deployment + "] in ZooKeeper since client session is closed"))
case e: Exception
handleError(new DeploymentException(
"Could not store deployment data [" + deployment + "] in ZooKeeper due to: " + e))
}
}
}
}
private def markDeploymentCompletedInCluster() {
ignore[ZkNodeExistsException](zkClient.create(isDeploymentCompletedInClusterLockPath, null, CreateMode.PERSISTENT))
}
private def isDeploymentCompletedInCluster = zkClient.exists(isDeploymentCompletedInClusterLockPath)
// FIXME in future - add watch to this path to be able to trigger redeployment, and use this method to trigger redeployment
private def invalidateDeploymentInCluster() {
ignore[ZkNoNodeException](zkClient.delete(isDeploymentCompletedInClusterLockPath))
}
private def ensureRunning[T](body: T): T = {
if (isConnected.isOn) body
else throw new IllegalStateException("ClusterDeployer is not running")
}
private[akka] def handleError(e: Throwable): Nothing = {
EventHandler.error(e, this, e.toString)
throw e
}
}

View file

@ -0,0 +1,26 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import com.typesafe.config.Config
import akka.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.config.ConfigurationException
import scala.collection.JavaConverters._
import akka.actor.Address
import akka.actor.AddressExtractor
class ClusterSettings(val config: Config, val systemName: String) {
import config._
// cluster config section
val FailureDetectorThreshold = getInt("akka.cluster.failure-detector.threshold")
val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size")
val SeedNodeConnectionTimeout = Duration(config.getMilliseconds("akka.cluster.seed-node-connection-timeout"), MILLISECONDS)
val MaxTimeToRetryJoiningCluster = Duration(config.getMilliseconds("akka.cluster.max-time-to-retry-joining-cluster"), MILLISECONDS)
val InitialDelayForGossip = Duration(getMilliseconds("akka.cluster.gossip.initialDelay"), MILLISECONDS)
val GossipFrequency = Duration(getMilliseconds("akka.cluster.gossip.frequency"), MILLISECONDS)
val SeedNodes = Set.empty[Address] ++ getStringList("akka.cluster.seed-nodes").asScala.collect {
case AddressExtractor(addr) addr
}
}

View file

@ -0,0 +1,438 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.actor._
import akka.actor.Status._
import akka.remote._
import akka.event.Logging
import akka.dispatch.Await
import akka.pattern.ask
import akka.util._
import akka.config.ConfigurationException
import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean }
import java.util.concurrent.TimeUnit._
import java.util.concurrent.TimeoutException
import java.security.SecureRandom
import System.{ currentTimeMillis newTimestamp }
import scala.collection.immutable.{ Map, SortedSet }
import scala.annotation.tailrec
import com.google.protobuf.ByteString
/**
* Interface for member membership change listener.
*/
trait NodeMembershipChangeListener {
def memberConnected(member: Member)
def memberDisconnected(member: Member)
}
/**
* Base trait for all cluster messages. All ClusterMessage's are serializable.
*/
sealed trait ClusterMessage extends Serializable
/**
* Command to join the cluster.
*/
case object JoinCluster extends ClusterMessage
/**
* Represents the state of the cluster; cluster ring membership, ring convergence, meta data - all versioned by a vector clock.
*/
case class Gossip(
version: VectorClock = VectorClock(),
member: Address,
// sorted set of members with their status, sorted by name
members: SortedSet[Member] = SortedSet.empty[Member](Ordering.fromLessThan[Member](_.address.toString > _.address.toString)),
unavailableMembers: Set[Member] = Set.empty[Member],
// for ring convergence
seen: Map[Member, VectorClock] = Map.empty[Member, VectorClock],
// for handoff
//pendingChanges: Option[Vector[PendingPartitioningChange]] = None,
meta: Option[Map[String, Array[Byte]]] = None)
extends ClusterMessage // is a serializable cluster message
with Versioned // has a vector clock as version
/**
* Represents the address and the current status of a cluster member node.
*/
case class Member(address: Address, status: MemberStatus) extends ClusterMessage
/**
* Defines the current status of a cluster member node
*
* Can be one of: Joining, Up, Leaving, Exiting and Down.
*/
sealed trait MemberStatus extends ClusterMessage with Versioned
object MemberStatus {
case class Joining(version: VectorClock = VectorClock()) extends MemberStatus
case class Up(version: VectorClock = VectorClock()) extends MemberStatus
case class Leaving(version: VectorClock = VectorClock()) extends MemberStatus
case class Exiting(version: VectorClock = VectorClock()) extends MemberStatus
case class Down(version: VectorClock = VectorClock()) extends MemberStatus
}
// sealed trait PendingPartitioningStatus
// object PendingPartitioningStatus {
// case object Complete extends PendingPartitioningStatus
// case object Awaiting extends PendingPartitioningStatus
// }
// case class PendingPartitioningChange(
// owner: Address,
// nextOwner: Address,
// changes: Vector[VNodeMod],
// status: PendingPartitioningStatus)
final class ClusterDaemon(system: ActorSystem, gossiper: Gossiper) extends Actor {
val log = Logging(system, "ClusterDaemon")
def receive = {
case JoinCluster sender ! gossiper.latestGossip
case gossip: Gossip
gossiper.tell(gossip)
case unknown log.error("Unknown message sent to cluster daemon [" + unknown + "]")
}
}
/**
* This module is responsible for Gossiping cluster information. The abstraction maintains the list of live
* and dead members. Periodically i.e. every 1 second this module chooses a random member and initiates a round
* of Gossip with it. Whenever it gets gossip updates it updates the Failure Detector with the liveness
* information.
* <p/>
* During each of these runs the member initiates gossip exchange according to following rules (as defined in the
* Cassandra documentation [http://wiki.apache.org/cassandra/ArchitectureGossip]:
* <pre>
* 1) Gossip to random live member (if any)
* 2) Gossip to random unreachable member with certain probability depending on number of unreachable and live members
* 3) If the member gossiped to at (1) was not seed, or the number of live members is less than number of seeds,
* gossip to random seed with certain probability depending on number of unreachable, seed and live members.
* </pre>
*/
case class Gossiper(remote: RemoteActorRefProvider, system: ActorSystemImpl) {
/**
* Represents the state for this Gossiper. Implemented using optimistic lockless concurrency,
* all state is represented by this immutable case class and managed by an AtomicReference.
*/
private case class State(
currentGossip: Gossip,
memberMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener])
val remoteSettings = new RemoteSettings(system.settings.config, system.name)
val clusterSettings = new ClusterSettings(system.settings.config, system.name)
val protocol = "akka" // TODO should this be hardcoded?
val address = remote.transport.address
val memberFingerprint = address.##
val initialDelayForGossip = clusterSettings.InitialDelayForGossip
val gossipFrequency = clusterSettings.GossipFrequency
implicit val seedNodeConnectionTimeout = clusterSettings.SeedNodeConnectionTimeout
implicit val defaultTimeout = Timeout(remoteSettings.RemoteSystemDaemonAckTimeout)
// seed members
private val seeds: Set[Member] = {
if (clusterSettings.SeedNodes.isEmpty) throw new ConfigurationException(
"At least one seed member must be defined in the configuration [akka.cluster.seed-members]")
else clusterSettings.SeedNodes map (address Member(address, MemberStatus.Up()))
}
private val serialization = remote.serialization
private val failureDetector = new AccrualFailureDetector(system, clusterSettings.FailureDetectorThreshold, clusterSettings.FailureDetectorMaxSampleSize)
private val isRunning = new AtomicBoolean(true)
private val log = Logging(system, "Gossiper")
private val random = SecureRandom.getInstance("SHA1PRNG")
// Is it right to put this guy under the /system path or should we have a top-level /cluster or something else...?
private val clusterDaemon = system.systemActorOf(Props(new ClusterDaemon(system, this)), "cluster")
private val state = new AtomicReference[State](State(currentGossip = newGossip()))
// FIXME manage connections in some other way so we can delete the RemoteConnectionManager (SINCE IT SUCKS!!!)
private val connectionManager = new RemoteConnectionManager(system, remote, failureDetector, Map.empty[Address, ActorRef])
log.info("Starting cluster Gossiper...")
// join the cluster by connecting to one of the seed members and retrieve current cluster state (Gossip)
joinCluster(clusterSettings.MaxTimeToRetryJoiningCluster fromNow)
// start periodic gossip and cluster scrutinization
val initateGossipCanceller = system.scheduler.schedule(initialDelayForGossip, gossipFrequency)(initateGossip())
val scrutinizeCanceller = system.scheduler.schedule(initialDelayForGossip, gossipFrequency)(scrutinize())
/**
* Shuts down all connections to other members, the cluster daemon and the periodic gossip and cleanup tasks.
*/
def shutdown() {
if (isRunning.compareAndSet(true, false)) {
log.info("Shutting down Gossiper for [{}]...", address)
try connectionManager.shutdown() finally {
try system.stop(clusterDaemon) finally {
try initateGossipCanceller.cancel() finally {
try scrutinizeCanceller.cancel() finally {
log.info("Gossiper for [{}] is shut down", address)
}
}
}
}
}
}
def latestGossip: Gossip = state.get.currentGossip
/**
* Tell the gossiper some gossip.
*/
//@tailrec
final def tell(newGossip: Gossip) {
val gossipingNode = newGossip.member
failureDetector heartbeat gossipingNode // update heartbeat in failure detector
// FIXME all below here is WRONG - redesign with cluster convergence in mind
// val oldState = state.get
// println("-------- NEW VERSION " + newGossip)
// println("-------- OLD VERSION " + oldState.currentGossip)
// val latestGossip = VectorClock.latestVersionOf(newGossip, oldState.currentGossip)
// println("-------- WINNING VERSION " + latestGossip)
// val latestAvailableNodes = latestGossip.members
// val latestUnavailableNodes = latestGossip.unavailableMembers
// println("=======>>> gossipingNode: " + gossipingNode)
// println("=======>>> latestAvailableNodes: " + latestAvailableNodes)
// if (!(latestAvailableNodes contains gossipingNode) && !(latestUnavailableNodes contains gossipingNode)) {
// println("-------- NEW NODE")
// // we have a new member
// val newGossip = latestGossip copy (availableNodes = latestAvailableNodes + gossipingNode)
// val newState = oldState copy (currentGossip = incrementVersionForGossip(newGossip))
// println("--------- new GOSSIP " + newGossip.members)
// println("--------- new STATE " + newState)
// // if we won the race then update else try again
// if (!state.compareAndSet(oldState, newState)) tell(newGossip) // recur
// else {
// println("---------- WON RACE - setting state")
// // create connections for all new members in the latest gossip
// (latestAvailableNodes + gossipingNode) foreach { member
// setUpConnectionToNode(member)
// oldState.memberMembershipChangeListeners foreach (_ memberConnected member) // notify listeners about the new members
// }
// }
// } else if (latestUnavailableNodes contains gossipingNode) {
// // gossip from an old former dead member
// val newUnavailableMembers = latestUnavailableNodes - gossipingNode
// val newMembers = latestAvailableNodes + gossipingNode
// val newGossip = latestGossip copy (availableNodes = newMembers, unavailableNodes = newUnavailableMembers)
// val newState = oldState copy (currentGossip = incrementVersionForGossip(newGossip))
// // if we won the race then update else try again
// if (!state.compareAndSet(oldState, newState)) tell(newGossip) // recur
// else oldState.memberMembershipChangeListeners foreach (_ memberConnected gossipingNode) // notify listeners on successful update of state
// }
}
/**
* Registers a listener to subscribe to cluster membership changes.
*/
@tailrec
final def registerListener(listener: NodeMembershipChangeListener) {
val oldState = state.get
val newListeners = oldState.memberMembershipChangeListeners + listener
val newState = oldState copy (memberMembershipChangeListeners = newListeners)
if (!state.compareAndSet(oldState, newState)) registerListener(listener) // recur
}
/**
* Unsubscribes to cluster membership changes.
*/
@tailrec
final def unregisterListener(listener: NodeMembershipChangeListener) {
val oldState = state.get
val newListeners = oldState.memberMembershipChangeListeners - listener
val newState = oldState copy (memberMembershipChangeListeners = newListeners)
if (!state.compareAndSet(oldState, newState)) unregisterListener(listener) // recur
}
/**
* Sets up remote connections to all the members in the argument list.
*/
private def connectToNodes(members: Seq[Member]) {
members foreach { member
setUpConnectionToNode(member)
state.get.memberMembershipChangeListeners foreach (_ memberConnected member) // notify listeners about the new members
}
}
// FIXME should shuffle list randomly before start traversing to avoid connecting to some member on every member
@tailrec
final private def connectToRandomNodeOf(members: Seq[Member]): ActorRef = {
members match {
case member :: rest
setUpConnectionToNode(member) match {
case Some(connection) connection
case None connectToRandomNodeOf(rest) // recur if
}
case Nil
throw new RemoteConnectionException(
"Could not establish connection to any of the members in the argument list")
}
}
/**
* Joins the cluster by connecting to one of the seed members and retrieve current cluster state (Gossip).
*/
private def joinCluster(deadline: Deadline) {
val seedNodes = seedNodesWithoutMyself // filter out myself
if (!seedNodes.isEmpty) { // if we have seed members to contact
connectToNodes(seedNodes)
try {
log.info("Trying to join cluster through one of the seed members [{}]", seedNodes.mkString(", "))
Await.result(connectToRandomNodeOf(seedNodes) ? JoinCluster, seedNodeConnectionTimeout) match {
case initialGossip: Gossip
// just sets/overwrites the state/gossip regardless of what it was before
// since it should be treated as the initial state
state.set(state.get copy (currentGossip = initialGossip))
log.debug("Received initial gossip [{}] from seed member", initialGossip)
case unknown
throw new IllegalStateException("Expected initial gossip from seed, received [" + unknown + "]")
}
} catch {
case e: Exception
log.error(
"Could not join cluster through any of the seed members - retrying for another {} seconds",
deadline.timeLeft.toSeconds)
// retry joining the cluster unless
// 1. Gossiper is shut down
// 2. The connection time window has expired
if (isRunning.get) {
if (deadline.timeLeft.toMillis > 0) joinCluster(deadline) // recur
else throw new RemoteConnectionException(
"Could not join cluster (any of the seed members) - giving up after trying for " +
deadline.time.toSeconds + " seconds")
}
}
}
}
/**
* Initates a new round of gossip.
*/
private def initateGossip() {
val oldState = state.get
val oldGossip = oldState.currentGossip
val oldMembers = oldGossip.members
val oldMembersSize = oldMembers.size
val oldUnavailableMembers = oldGossip.unavailableMembers
val oldUnavailableMembersSize = oldUnavailableMembers.size
// 1. gossip to alive members
val gossipedToSeed =
if (oldUnavailableMembersSize > 0) gossipToRandomNodeOf(oldMembers)
else false
// 2. gossip to dead members
if (oldUnavailableMembersSize > 0) {
val probability: Double = oldUnavailableMembersSize / (oldMembersSize + 1)
if (random.nextDouble() < probability) gossipToRandomNodeOf(oldUnavailableMembers)
}
// 3. gossip to a seed for facilitating partition healing
if ((!gossipedToSeed || oldMembersSize < 1) && (seeds.head != address)) {
if (oldMembersSize == 0) gossipToRandomNodeOf(seeds)
else {
val probability = 1.0 / oldMembersSize + oldUnavailableMembersSize
if (random.nextDouble() <= probability) gossipToRandomNodeOf(seeds)
}
}
}
/**
* Gossips to a random member in the set of members passed in as argument.
*
* @returns 'true' if it gossiped to a "seed" member.
*/
private def gossipToRandomNodeOf(members: Set[Member]): Boolean = {
val peers = members filter (_.address != address) // filter out myself
val peer = selectRandomNode(peers)
val oldState = state.get
val oldGossip = oldState.currentGossip
// if connection can't be established/found => ignore it since the failure detector will take care of the potential problem
setUpConnectionToNode(peer) foreach { _ ! newGossip }
seeds exists (peer == _)
}
/**
* Scrutinizes the cluster; marks members detected by the failure detector as unavailable, and notifies all listeners
* of the change in the cluster membership.
*/
@tailrec
final private def scrutinize() {
val oldState = state.get
val oldGossip = oldState.currentGossip
val oldMembers = oldGossip.members
val oldUnavailableMembers = oldGossip.unavailableMembers
val newlyDetectedUnavailableMembers = oldMembers filterNot (member failureDetector.isAvailable(member.address))
if (!newlyDetectedUnavailableMembers.isEmpty) { // we have newly detected members marked as unavailable
val newMembers = oldMembers diff newlyDetectedUnavailableMembers
val newUnavailableMembers = oldUnavailableMembers ++ newlyDetectedUnavailableMembers
val newGossip = oldGossip copy (members = newMembers, unavailableMembers = newUnavailableMembers)
val newState = oldState copy (currentGossip = incrementVersionForGossip(newGossip))
// if we won the race then update else try again
if (!state.compareAndSet(oldState, newState)) scrutinize() // recur
else {
// notify listeners on successful update of state
for {
deadNode newUnavailableMembers
listener oldState.memberMembershipChangeListeners
} listener memberDisconnected deadNode
}
}
}
private def setUpConnectionToNode(member: Member): Option[ActorRef] = {
val address = member.address
try {
Some(
connectionManager.putIfAbsent(
address,
() system.actorFor(RootActorPath(Address(protocol, system.name)) / "system" / "cluster")))
} catch {
case e: Exception None
}
}
private def newGossip(): Gossip = Gossip(member = address)
private def incrementVersionForGossip(from: Gossip): Gossip = {
val newVersion = from.version.increment(memberFingerprint, newTimestamp)
from copy (version = newVersion)
}
private def seedNodesWithoutMyself: List[Member] = seeds.filter(_.address != address).toList
private def selectRandomNode(members: Set[Member]): Member = members.toList(random.nextInt(members.size))
}

View file

@ -1,105 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster
import akka.config.Config
import Config._
import akka.util._
import Helpers._
import akka.actor._
import Actor._
import akka.event.EventHandler
import akka.cluster.zookeeper._
import org.apache.zookeeper._
import org.apache.zookeeper.Watcher.Event._
import org.apache.zookeeper.data.Stat
import org.apache.zookeeper.recipes.lock.{ WriteLock, LockListener }
import org.I0Itec.zkclient._
import org.I0Itec.zkclient.serialize._
import org.I0Itec.zkclient.exception._
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference }
object LocalCluster {
val clusterDirectory = config.getString("akka.cluster.log-directory", "_akka_cluster")
val clusterDataDirectory = clusterDirectory + "/data"
val clusterLogDirectory = clusterDirectory + "/log"
val clusterName = Config.clusterName
val nodename = Config.nodename
val zooKeeperServers = config.getString("akka.cluster.zookeeper-server-addresses", "localhost:2181")
val sessionTimeout = Duration(config.getInt("akka.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt
val connectionTimeout = Duration(config.getInt("akka.cluster.connection-timeout", 60), TIME_UNIT).toMillis.toInt
val defaultZooKeeperSerializer = new SerializableSerializer
val zkServer = new AtomicReference[Option[ZkServer]](None)
lazy val zkClient = new AkkaZkClient(zooKeeperServers, sessionTimeout, connectionTimeout, defaultZooKeeperSerializer)
/**
* Looks up the local hostname.
*/
def lookupLocalhostName = NetworkUtil.getLocalhostName
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalCluster(): ZkServer =
startLocalCluster(clusterDataDirectory, clusterLogDirectory, 2181, 5000)
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalCluster(port: Int, tickTime: Int): ZkServer =
startLocalCluster(clusterDataDirectory, clusterLogDirectory, port, tickTime)
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalCluster(tickTime: Int): ZkServer =
startLocalCluster(clusterDataDirectory, clusterLogDirectory, 2181, tickTime)
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalCluster(dataPath: String, logPath: String): ZkServer =
startLocalCluster(dataPath, logPath, 2181, 500)
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalCluster(dataPath: String, logPath: String, port: Int, tickTime: Int): ZkServer = {
try {
val zk = AkkaZooKeeper.startLocalServer(dataPath, logPath, port, tickTime)
zkServer.set(Some(zk))
zk
} catch {
case e: Throwable
EventHandler.error(e, this, "Could not start local ZooKeeper cluster")
throw e
}
}
/**
* Shut down the local ZooKeeper server.
*/
def shutdownLocalCluster() {
withPrintStackTraceOnError {
EventHandler.debug(this, "Shuts down local cluster")
zkServer.getAndSet(None).foreach(_.shutdown())
}
}
def createQueue(rootPath: String, blocking: Boolean = true) =
new ZooKeeperQueue(zkClient, rootPath, blocking)
def barrier(name: String, count: Int): ZooKeeperBarrier =
ZooKeeperBarrier(zkClient, clusterName, name, nodename, count)
def barrier(name: String, count: Int, timeout: Duration): ZooKeeperBarrier =
ZooKeeperBarrier(zkClient, clusterName, name, nodename, count, timeout)
}

View file

@ -2,9 +2,10 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
package akka.cluster
import akka.actor._
import akka.remote._
import akka.routing._
import akka.event.Logging
@ -19,6 +20,7 @@ import java.util.concurrent.atomic.AtomicReference
class RemoteConnectionManager(
system: ActorSystemImpl,
remote: RemoteActorRefProvider,
failureDetector: AccrualFailureDetector,
initialConnections: Map[Address, ActorRef] = Map.empty[Address, ActorRef])
extends ConnectionManager {
@ -30,8 +32,6 @@ class RemoteConnectionManager(
def iterable: Iterable[ActorRef] = connections.values
}
def failureDetector = remote.failureDetector
private val state: AtomicReference[State] = new AtomicReference[State](newState())
/**
@ -145,6 +145,6 @@ class RemoteConnectionManager(
}
}
private[remote] def newConnection(remoteAddress: Address, actorPath: ActorPath) =
private[cluster] def newConnection(remoteAddress: Address, actorPath: ActorPath) =
new RemoteActorRef(remote, remote.transport, actorPath, Nobody)
}

View file

@ -1,604 +0,0 @@
package akka.cluster
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
import org.apache.bookkeeper.client.{ BookKeeper, LedgerHandle, LedgerEntry, BKException, AsyncCallback }
import org.apache.zookeeper.CreateMode
import org.I0Itec.zkclient.exception._
import akka.AkkaException
import akka.config._
import Config._
import akka.util._
import akka.actor._
import DeploymentConfig.ReplicationScheme
import akka.event.EventHandler
import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation }
import akka.cluster.zookeeper._
import akka.serialization.ActorSerialization._
import akka.serialization.Compression.LZF
import java.util.Enumeration
// FIXME allow user to choose dynamically between 'async' and 'sync' tx logging (asyncAddEntry(byte[] data, AddCallback cb, Object ctx))
// FIXME clean up old entries in log after doing a snapshot
class ReplicationException(message: String, cause: Throwable = null) extends AkkaException(message) {
def this(msg: String) = this(msg, null)
}
/**
* A TransactionLog makes chunks of data durable.
*/
class TransactionLog private (
ledger: LedgerHandle,
val id: String,
val isAsync: Boolean,
replicationScheme: ReplicationScheme) {
import TransactionLog._
val logId = ledger.getId
val txLogPath = transactionLogPath(id)
val snapshotPath = txLogPath + "/snapshot"
private val isOpen = new Switch(true)
/**
* Record an Actor message invocation.
*
* @param invocation the MessageInvocation to record
* @param actorRef the LocalActorRef that received the message.
* @throws ReplicationException if the TransactionLog already is closed.
*/
def recordEntry(invocation: MessageInvocation, actorRef: LocalActorRef) {
val entryId = ledger.getLastAddPushed + 1
val needsSnapshot = entryId != 0 && (entryId % snapshotFrequency) == 0
if (needsSnapshot) {
//todo: could it be that the message is never persisted when a snapshot is added?
val bytes = toBinary(actorRef, false, replicationScheme)
recordSnapshot(bytes)
} else {
val bytes = MessageSerializer.serialize(invocation.message.asInstanceOf[AnyRef]).toByteArray
recordEntry(bytes)
}
}
/**
* Record an entry.
*
* @param entry the entry in byte form to record.
* @throws ReplicationException if the TransactionLog already is closed.
*/
def recordEntry(entry: Array[Byte]) {
if (isOpen.isOn) {
val entryBytes =
if (shouldCompressData) LZF.compress(entry)
else entry
try {
if (isAsync) {
ledger.asyncAddEntry(
entryBytes,
new AsyncCallback.AddCallback {
def addComplete(returnCode: Int, ledgerHandle: LedgerHandle, entryId: Long, ctx: AnyRef) {
handleReturnCode(returnCode)
EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId))
}
},
null)
} else {
handleReturnCode(ledger.addEntry(entryBytes))
val entryId = ledger.getLastAddPushed
EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId))
}
} catch {
case e: Throwable handleError(e)
}
} else transactionClosedError
}
/**
* Record a snapshot.
*
* @param snapshot the snapshot in byteform to record.
* @throws ReplicationException if the TransactionLog already is closed.
*/
def recordSnapshot(snapshot: Array[Byte]) {
if (isOpen.isOn) {
val snapshotBytes =
if (shouldCompressData) LZF.compress(snapshot)
else snapshot
try {
if (isAsync) {
ledger.asyncAddEntry(
snapshotBytes,
new AsyncCallback.AddCallback {
def addComplete(returnCode: Int, ledgerHandle: LedgerHandle, snapshotId: Long, ctx: AnyRef) {
handleReturnCode(returnCode)
EventHandler.debug(this, "Writing snapshot to log [%s]".format(snapshotId))
storeSnapshotMetaDataInZooKeeper(snapshotId)
}
},
null)
} else {
//todo: could this be racy, since writing the snapshot itself and storing the snapsnot id, is not
//an atomic operation?
//first store the snapshot.
handleReturnCode(ledger.addEntry(snapshotBytes))
val snapshotId = ledger.getLastAddPushed
//this is the location where all previous entries can be removed.
//TODO: how to remove data?
EventHandler.debug(this, "Writing snapshot to log [%s]".format(snapshotId))
//and now store the snapshot metadata.
storeSnapshotMetaDataInZooKeeper(snapshotId)
}
} catch {
case e: Throwable handleError(e)
}
} else transactionClosedError
}
/**
* Get all the entries for this transaction log.
*
* @throws ReplicationException if the TransactionLog already is closed.
*/
def entries: Vector[Array[Byte]] = entriesInRange(0, ledger.getLastAddConfirmed)
/**
* Get the latest snapshot and all subsequent entries from this snapshot.
*/
def latestSnapshotAndSubsequentEntries: (Option[Array[Byte]], Vector[Array[Byte]]) = {
latestSnapshotId match {
case Some(snapshotId)
EventHandler.debug(this, "Reading entries from snapshot id [%s] for log [%s]".format(snapshotId, logId))
val cursor = snapshotId + 1
val lastIndex = ledger.getLastAddConfirmed
val snapshot = Some(entriesInRange(snapshotId, snapshotId).head)
val entries =
if ((cursor - lastIndex) == 0) Vector.empty[Array[Byte]]
else entriesInRange(cursor, lastIndex)
(snapshot, entries)
case None
(None, entries)
}
}
/**
* Get a range of entries from 'from' to 'to' for this transaction log.
*
* @param from the first element of the range
* @param the last index from the range (including).
* @return a Vector containing Byte Arrays. Each element in the vector is a record.
* @throws IllegalArgumenException if from or to is negative, or if 'from' is bigger than 'to'.
* @throws ReplicationException if the TransactionLog already is closed.
*/
def entriesInRange(from: Long, to: Long): Vector[Array[Byte]] = if (isOpen.isOn) {
try {
if (from < 0) throw new IllegalArgumentException("'from' index can't be negative [" + from + "]")
if (to < 0) throw new IllegalArgumentException("'to' index can't be negative [" + from + "]")
if (to < from) throw new IllegalArgumentException("'to' index can't be smaller than 'from' index [" + from + "," + to + "]")
EventHandler.debug(this, "Reading entries [%s -> %s] for log [%s]".format(from, to, logId))
if (isAsync) {
val future = Promise[Vector[Array[Byte]]]()
ledger.asyncReadEntries(
from, to,
new AsyncCallback.ReadCallback {
def readComplete(returnCode: Int, ledgerHandle: LedgerHandle, enumeration: Enumeration[LedgerEntry], ctx: AnyRef) {
val future = ctx.asInstanceOf[Promise[Vector[Array[Byte]]]]
val entries = toByteArrays(enumeration)
if (returnCode == BKException.Code.OK) future.success(entries)
else future.failure(BKException.create(returnCode))
}
},
future)
await(future)
} else {
toByteArrays(ledger.readEntries(from, to))
}
} catch {
case e: Throwable handleError(e)
}
} else transactionClosedError
/**
* Get the last entry written to this transaction log.
*
* Returns -1 if there has never been an entry.
*/
def latestEntryId: Long = ledger.getLastAddConfirmed
/**
* Get the id for the last snapshot written to this transaction log.
*/
def latestSnapshotId: Option[Long] = {
try {
val snapshotId = zkClient.readData(snapshotPath).asInstanceOf[Long]
EventHandler.debug(this, "Retrieved latest snapshot id [%s] from transaction log [%s]".format(snapshotId, logId))
Some(snapshotId)
} catch {
case e: ZkNoNodeException None
case e: Throwable handleError(e)
}
}
/**
* Delete this transaction log. So all entries but also all metadata will be removed.
*
* TODO: Behavior unclear what happens when already deleted (what happens to the ledger).
* TODO: Behavior unclear what happens when already closed.
*/
def delete() {
if (isOpen.isOn) {
EventHandler.debug(this, "Deleting transaction log [%s]".format(logId))
try {
if (isAsync) {
bookieClient.asyncDeleteLedger(
logId,
new AsyncCallback.DeleteCallback {
def deleteComplete(returnCode: Int, ctx: AnyRef) {
(returnCode)
}
},
null)
} else {
bookieClient.deleteLedger(logId)
}
//also remote everything else that belongs to this TransactionLog.
zkClient.delete(snapshotPath)
zkClient.delete(txLogPath)
} catch {
case e: Throwable handleError(e)
}
}
}
/**
* Close this transaction log.
*
* If already closed, the call is ignored.
*/
def close() {
isOpen switchOff {
EventHandler.debug(this, "Closing transaction log [%s]".format(logId))
try {
if (isAsync) {
ledger.asyncClose(
new AsyncCallback.CloseCallback {
def closeComplete(
returnCode: Int,
ledgerHandle: LedgerHandle,
ctx: AnyRef) {
handleReturnCode(returnCode)
}
},
null)
} else {
ledger.close()
}
} catch {
case e: Throwable handleError(e)
}
}
}
private def toByteArrays(enumeration: Enumeration[LedgerEntry]): Vector[Array[Byte]] = {
var entries = Vector[Array[Byte]]()
while (enumeration.hasMoreElements) {
val bytes = enumeration.nextElement.getEntry
val entry =
if (shouldCompressData) LZF.uncompress(bytes)
else bytes
entries = entries :+ entry
}
entries
}
private def storeSnapshotMetaDataInZooKeeper(snapshotId: Long) {
if (isOpen.isOn) {
try {
zkClient.create(snapshotPath, null, CreateMode.PERSISTENT)
} catch {
case e: ZkNodeExistsException {} // do nothing
case e: Throwable handleError(e)
}
try {
zkClient.writeData(snapshotPath, snapshotId)
} catch {
case e: Throwable
handleError(new ReplicationException(
"Could not store transaction log snapshot meta-data in ZooKeeper for UUID [" + id + "]"))
}
EventHandler.debug(this, "Writing snapshot [%s] to log [%s]".format(snapshotId, logId))
} else transactionClosedError
}
private def handleReturnCode(block: Long) {
val code = block.toInt
if (code == BKException.Code.OK) {} // all fine
else handleError(BKException.create(code))
}
private def transactionClosedError: Nothing = {
handleError(new ReplicationException(
"Transaction log [" + logId +
"] is closed. You need to open up new a new one with 'TransactionLog.logFor(id)'"))
}
}
/**
* TODO: Documentation.
*/
object TransactionLog {
val zooKeeperServers = config.getString("akka.cluster.zookeeper-server-addresses", "localhost:2181")
val sessionTimeout = Duration(config.getInt("akka.cluster.session-timeout", 60), TIME_UNIT).toMillis.toInt
val connectionTimeout = Duration(config.getInt("akka.cluster.connection-timeout", 60), TIME_UNIT).toMillis.toInt
val digestType = config.getString("akka.cluster.replication.digest-type", "CRC32") match {
case "CRC32" BookKeeper.DigestType.CRC32
case "MAC" BookKeeper.DigestType.MAC
case unknown throw new ConfigurationException(
"akka.cluster.replication.digest-type is invalid [" + unknown + "], must be either 'CRC32' or 'MAC'")
}
val password = config.getString("akka.cluster.replication.password", "secret").getBytes("UTF-8")
val ensembleSize = config.getInt("akka.cluster.replication.ensemble-size", 3)
val quorumSize = config.getInt("akka.cluster.replication.quorum-size", 2)
val snapshotFrequency = config.getInt("akka.cluster.replication.snapshot-frequency", 1000)
val timeout = Duration(config.getInt("akka.cluster.replication.timeout", 30), TIME_UNIT).toMillis
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
private[akka] val transactionLogNode = "/transaction-log-ids"
private val isConnected = new Switch(false)
@volatile
private[akka] var bookieClient: BookKeeper = _
@volatile
private[akka] var zkClient: AkkaZkClient = _
private[akka] def apply(
ledger: LedgerHandle,
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme) =
new TransactionLog(ledger, id, isAsync, replicationScheme)
/**
* Starts up the transaction log.
*/
def start() {
isConnected switchOn {
bookieClient = new BookKeeper(zooKeeperServers)
zkClient = new AkkaZkClient(zooKeeperServers, sessionTimeout, connectionTimeout)
try {
zkClient.create(transactionLogNode, null, CreateMode.PERSISTENT)
} catch {
case e: ZkNodeExistsException {} // do nothing
case e: Throwable handleError(e)
}
EventHandler.info(this,
("Transaction log service started with" +
"\n\tdigest type [%s]" +
"\n\tensemble size [%s]" +
"\n\tquorum size [%s]" +
"\n\tlogging time out [%s]").format(
digestType,
ensembleSize,
quorumSize,
timeout))
}
}
/**
* Shuts down the transaction log.
*/
def shutdown() {
isConnected switchOff {
try {
EventHandler.info(this, "Shutting down transaction log...")
zkClient.close()
bookieClient.halt()
EventHandler.info(this, "Transaction log shut down successfully")
} catch {
case e: Throwable handleError(e)
}
}
}
def transactionLogPath(id: String): String = transactionLogNode + "/" + id
/**
* Checks if a TransactionLog for the given id already exists.
*/
def exists(id: String): Boolean = {
val txLogPath = transactionLogPath(id)
zkClient.exists(txLogPath)
}
/**
* Creates a new transaction log for the 'id' specified. If a TransactionLog already exists for the id,
* it will be overwritten.
*/
def newLogFor(id: String, isAsync: Boolean, replicationScheme: ReplicationScheme): TransactionLog = {
val txLogPath = transactionLogPath(id)
val ledger = try {
if (exists(id)) {
//if it exists, we need to delete it first. This gives it the overwrite semantics we are looking for.
try {
val ledger = bookieClient.createLedger(ensembleSize, quorumSize, digestType, password)
val txLog = TransactionLog(ledger, id, false, null)
txLog.delete()
txLog.close()
} catch {
case e: Throwable handleError(e)
}
}
val future = Promise[LedgerHandle]()
if (isAsync) {
bookieClient.asyncCreateLedger(
ensembleSize, quorumSize, digestType, password,
new AsyncCallback.CreateCallback {
def createComplete(
returnCode: Int,
ledgerHandle: LedgerHandle,
ctx: AnyRef) {
val future = ctx.asInstanceOf[Promise[LedgerHandle]]
if (returnCode == BKException.Code.OK) future.success(ledgerHandle)
else future.failure(BKException.create(returnCode))
}
},
future)
await(future)
} else {
bookieClient.createLedger(ensembleSize, quorumSize, digestType, password)
}
} catch {
case e: Throwable handleError(e)
}
val logId = ledger.getId
try {
zkClient.create(txLogPath, null, CreateMode.PERSISTENT)
zkClient.writeData(txLogPath, logId)
logId //TODO: does this have any effect?
} catch {
case e: Throwable
bookieClient.deleteLedger(logId) // clean up
handleError(new ReplicationException(
"Could not store transaction log [" + logId +
"] meta-data in ZooKeeper for UUID [" + id + "]", e))
}
EventHandler.info(this, "Created new transaction log [%s] for UUID [%s]".format(logId, id))
TransactionLog(ledger, id, isAsync, replicationScheme)
}
/**
* Fetches an existing transaction log for the 'id' specified.
*
* @throws ReplicationException if the log with the given id doesn't exist.
*/
def logFor(id: String, isAsync: Boolean, replicationScheme: ReplicationScheme): TransactionLog = {
val txLogPath = transactionLogPath(id)
val logId = try {
val logId = zkClient.readData(txLogPath).asInstanceOf[Long]
EventHandler.debug(this,
"Retrieved transaction log [%s] for UUID [%s]".format(logId, id))
logId
} catch {
case e: ZkNoNodeException
handleError(new ReplicationException(
"Transaction log for UUID [" + id + "] does not exist in ZooKeeper"))
case e: Throwable handleError(e)
}
val ledger = try {
if (isAsync) {
val future = Promise[LedgerHandle]()
bookieClient.asyncOpenLedger(
logId, digestType, password,
new AsyncCallback.OpenCallback {
def openComplete(returnCode: Int, ledgerHandle: LedgerHandle, ctx: AnyRef) {
val future = ctx.asInstanceOf[Promise[LedgerHandle]]
if (returnCode == BKException.Code.OK) future.success(ledgerHandle)
else future.failure(BKException.create(returnCode))
}
},
future)
await(future)
} else {
bookieClient.openLedger(logId, digestType, password)
}
} catch {
case e: Throwable handleError(e)
}
TransactionLog(ledger, id, isAsync, replicationScheme)
}
private[akka] def await[T](future: Promise[T]): T = {
future.await.value.get match {
case Right(result) => result
case Left(throwable) => handleError(throwable)
}
}
private[akka] def handleError(e: Throwable): Nothing = {
EventHandler.error(e, this, e.toString)
throw e
}
}
/**
* TODO: Documentation.
*/
object LocalBookKeeperEnsemble {
private val isRunning = new Switch(false)
//TODO: should probably come from the config file.
private val port = 5555
@volatile
private var localBookKeeper: LocalBookKeeper = _
/**
* Starts the LocalBookKeeperEnsemble.
*
* Call can safely be made when already started.
*
* This call will block until it is started.
*/
def start() {
isRunning switchOn {
EventHandler.info(this, "Starting up LocalBookKeeperEnsemble...")
localBookKeeper = new LocalBookKeeper(TransactionLog.ensembleSize)
localBookKeeper.runZookeeper(port)
localBookKeeper.initializeZookeper()
localBookKeeper.runBookies()
EventHandler.info(this, "LocalBookKeeperEnsemble started up successfully")
}
}
/**
* Shuts down the LocalBookKeeperEnsemble.
*
* Call can safely bemade when already shutdown.
*
* This call will block until the shutdown completes.
*/
def shutdown() {
isRunning switchOff {
EventHandler.info(this, "Shutting down LocalBookKeeperEnsemble...")
localBookKeeper.bs.foreach(_.shutdown()) // stop bookies
localBookKeeper.zkc.close() // stop zk client
localBookKeeper.zks.shutdown() // stop zk server
localBookKeeper.serverFactory.shutdown() // stop zk NIOServer
EventHandler.info(this, "LocalBookKeeperEnsemble shut down successfully")
}
}
}

View file

@ -2,18 +2,39 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
package akka.cluster
import akka.AkkaException
class VectorClockException(message: String) extends AkkaException(message)
/**
* Trait to be extended by classes that wants to be versioned using a VectorClock.
*/
trait Versioned {
def version: VectorClock
}
/**
* Utility methods for comparing Versioned instances.
*/
object Versioned {
def latestVersionOf[T <: Versioned](versioned1: T, versioned2: T): T = {
(versioned1.version compare versioned2.version) match {
case VectorClock.Before versioned2 // version 1 is BEFORE (older), use version 2
case VectorClock.After versioned1 // version 1 is AFTER (newer), use version 1
case VectorClock.Concurrent versioned1 // can't establish a causal relationship between versions => conflict - keeping version 1
}
}
}
/**
* Representation of a Vector-based clock (counting clock), inspired by Lamport logical clocks.
*
* {{
* Reference:
* Leslie Lamport (1978). "Time, clocks, and the ordering of events in a distributed system". Communications of the ACM 21 (7): 558-565.
* Friedemann Mattern (1988). "Virtual Time and Global States of Distributed Systems". Workshop on Parallel and Distributed Algorithms: pp. 215-226
* 1) Leslie Lamport (1978). "Time, clocks, and the ordering of events in a distributed system". Communications of the ACM 21 (7): 558-565.
* 2) Friedemann Mattern (1988). "Virtual Time and Global States of Distributed Systems". Workshop on Parallel and Distributed Algorithms: pp. 215-226
* }}
*/
case class VectorClock(
versions: Vector[VectorClock.Entry] = Vector.empty[VectorClock.Entry],
@ -55,9 +76,11 @@ object VectorClock {
/**
* The result of comparing two vector clocks.
* Either:
* {{
* 1) v1 is BEFORE v2
* 2) v1 is AFTER t2
* 3) v1 happens CONCURRENTLY to v2
* }}
*/
sealed trait Ordering
case object Before extends Ordering
@ -74,9 +97,11 @@ object 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).
* 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.
* }}
*
* @param v1 The first VectorClock
* @param v2 The second VectorClock

View file

@ -1,226 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.cluster._
import Cluster._
import akka.cluster.zookeeper._
import akka.actor._
import Actor._
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
import java.util.concurrent.{ ConcurrentHashMap, ConcurrentSkipListSet }
import java.util.concurrent.atomic.AtomicReference
import akka.util.{ Duration, Switch }
import akka.util.Helpers._
import akka.util.duration._
import org.I0Itec.zkclient.exception.ZkNoNodeException
import akka.event.EventHandler
/*
* Instance of the metrics manager running on the node. To keep the fine performance, metrics of all the
* nodes in the cluster are cached internally, and refreshed from monitoring MBeans / Sigar (when if's local node),
* of ZooKeeper (if it's metrics of all the nodes in the cluster) after a specified timeout -
* <code>metricsRefreshTimeout</code>
* <code>metricsRefreshTimeout</code> defaults to 2 seconds, and can be declaratively defined through
* akka.conf:
*
* @exampl {{{
* akka.cluster.metrics-refresh-timeout = 2
* }}}
*/
class LocalNodeMetricsManager(zkClient: AkkaZkClient, private val metricsRefreshTimeout: Duration)
extends NodeMetricsManager {
/*
* Provides metrics of the system that the node is running on, through monitoring MBeans, Hyperic Sigar
* and other systems
*/
lazy private val metricsProvider = SigarMetricsProvider(refreshTimeout.toMillis.toInt) fold ((thrw) {
EventHandler.warning(this, """Hyperic Sigar library failed to load due to %s: %s.
All the metrics will be retreived from monitoring MBeans, and may be incorrect at some platforms.
In order to get better metrics, please put "sigar.jar" to the classpath, and add platform-specific native libary to "java.library.path"."""
.format(thrw.getClass.getName, thrw.getMessage))
new JMXMetricsProvider
},
sigar sigar)
/*
* Metrics of all nodes in the cluster
*/
private val localNodeMetricsCache = new ConcurrentHashMap[String, NodeMetrics]
@volatile
private var _refreshTimeout = metricsRefreshTimeout
/*
* Plugged monitors (both local and cluster-wide)
*/
private val alterationMonitors = new ConcurrentSkipListSet[MetricsAlterationMonitor]
private val _isRunning = new Switch(false)
/*
* If the value is <code>true</code>, metrics manages is started and running. Stopped, otherwise
*/
def isRunning = _isRunning.isOn
/*
* Starts metrics manager. When metrics manager is started, it refreshes cache from ZooKeeper
* after <code>refreshTimeout</code>, and invokes plugged monitors
*/
def start() = {
_isRunning.switchOn { refresh() }
this
}
private[cluster] def metricsForNode(nodeName: String): String = "%s/%s".format(node.NODE_METRICS, nodeName)
/*
* Adds monitor that reacts, when specific conditions are satisfied
*/
def addMonitor(monitor: MetricsAlterationMonitor) = alterationMonitors add monitor
def removeMonitor(monitor: MetricsAlterationMonitor) = alterationMonitors remove monitor
def refreshTimeout_=(newValue: Duration) = _refreshTimeout = newValue
/*
* Timeout after which metrics, cached in the metrics manager, will be refreshed from ZooKeeper
*/
def refreshTimeout = _refreshTimeout
/*
* Stores metrics of the node in ZooKeeper
*/
private[akka] def storeMetricsInZK(metrics: NodeMetrics) = {
val metricsPath = metricsForNode(metrics.nodeName)
if (zkClient.exists(metricsPath)) {
zkClient.writeData(metricsPath, metrics)
} else {
ignore[ZkNoNodeException](zkClient.createEphemeral(metricsPath, metrics))
}
}
/*
* Gets metrics of the node from ZooKeeper
*/
private[akka] def getMetricsFromZK(nodeName: String) = {
zkClient.readData[NodeMetrics](metricsForNode(nodeName))
}
/*
* Removed metrics of the node from local cache and ZooKeeper
*/
def removeNodeMetrics(nodeName: String) = {
val metricsPath = metricsForNode(nodeName)
if (zkClient.exists(metricsPath)) {
ignore[ZkNoNodeException](zkClient.delete(metricsPath))
}
localNodeMetricsCache.remove(nodeName)
}
/*
* Gets metrics of a local node directly from JMX monitoring beans/Hyperic Sigar
*/
def getLocalMetrics = metricsProvider.getLocalMetrics
/*
* Gets metrics of the node, specified by the name. If <code>useCached</code> is true (default value),
* metrics snapshot is taken from the local cache; otherwise, it's retreived from ZooKeeper'
*/
def getMetrics(nodeName: String, useCached: Boolean = true): Option[NodeMetrics] =
if (useCached)
Option(localNodeMetricsCache.get(nodeName))
else
try {
Some(getMetricsFromZK(nodeName))
} catch {
case ex: ZkNoNodeException None
}
/*
* Return metrics of all nodes in the cluster from ZooKeeper
*/
private[akka] def getAllMetricsFromZK: Map[String, NodeMetrics] = {
val metricsPaths = zkClient.getChildren(node.NODE_METRICS).toList.toArray.asInstanceOf[Array[String]]
metricsPaths.flatMap { nodeName getMetrics(nodeName, false).map((nodeName, _)) } toMap
}
/*
* Gets cached metrics of all nodes in the cluster
*/
def getAllMetrics: Array[NodeMetrics] = localNodeMetricsCache.values.asScala.toArray
/*
* Refreshes locally cached metrics from ZooKeeper, and invokes plugged monitors
*/
private[akka] def refresh() {
storeMetricsInZK(getLocalMetrics)
refreshMetricsCacheFromZK()
if (isRunning) {
Scheduler.schedule({ () refresh() }, refreshTimeout.length, refreshTimeout.length, refreshTimeout.unit)
invokeMonitors()
}
}
/*
* Refreshes metrics manager cache from ZooKeeper
*/
private def refreshMetricsCacheFromZK() {
val allMetricsFromZK = getAllMetricsFromZK
localNodeMetricsCache.keySet.foreach { key
if (!allMetricsFromZK.contains(key))
localNodeMetricsCache.remove(key)
}
// RACY: metrics for the node might have been removed both from ZK and local cache by the moment,
// but will be re-cached, since they're still present in allMetricsFromZK snapshot. Not important, because
// cache will be fixed soon, at the next iteration of refresh
allMetricsFromZK map {
case (node, metrics)
localNodeMetricsCache.put(node, metrics)
}
}
/*
* Invokes monitors with the cached metrics
*/
private def invokeMonitors(): Unit = if (!alterationMonitors.isEmpty) {
// RACY: metrics for some nodes might have been removed/added by that moment. Not important,
// because monitors will be fed with up-to-date metrics shortly, at the next iteration of refresh
val clusterNodesMetrics = getAllMetrics
val localNodeMetrics = clusterNodesMetrics.find(_.nodeName == nodeAddress.nodeName)
val iterator = alterationMonitors.iterator
// RACY: there might be new monitors added after the iterator has been obtained. Not important,
// becuse refresh interval is meant to be very short, and all the new monitors will be called ad the
// next refresh iteration
while (iterator.hasNext) {
val monitor = iterator.next
monitor match {
case localMonitor: LocalMetricsAlterationMonitor
localNodeMetrics.map { metrics
if (localMonitor reactsOn metrics)
localMonitor react metrics
}
case clusterMonitor: ClusterMetricsAlterationMonitor
if (clusterMonitor reactsOn clusterNodesMetrics)
clusterMonitor react clusterNodesMetrics
}
}
}
def stop() = _isRunning.switchOff
}

View file

@ -1,154 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.cluster._
import akka.event.EventHandler
import java.lang.management.ManagementFactory
import akka.util.ReflectiveAccess._
import akka.util.Switch
/*
* Snapshot of the JVM / system that's the node is running on
*
* @param nodeName name of the node, where metrics are gathered at
* @param usedHeapMemory amount of heap memory currently used
* @param committedHeapMemory amount of heap memory guaranteed to be available
* @param maxHeapMemory maximum amount of heap memory that can be used
* @param avaiableProcessors number of the processors avalable to the JVM
* @param systemLoadAverage system load average. If OS-specific Sigar's native library is plugged,
* it's used to calculate average load on the CPUs in the system. Otherwise, value is retreived from monitoring
* MBeans. Hyperic Sigar provides more precise values, and, thus, if the library is provided, it's used by default.
*
*/
case class DefaultNodeMetrics(nodeName: String,
usedHeapMemory: Long,
committedHeapMemory: Long,
maxHeapMemory: Long,
avaiableProcessors: Int,
systemLoadAverage: Double) extends NodeMetrics
object MetricsProvider {
/*
* Maximum value of system load average
*/
val MAX_SYS_LOAD_AVG = 1
/*
* Minimum value of system load average
*/
val MIN_SYS_LOAD_AVG = 0
/*
* Default value of system load average
*/
val DEF_SYS_LOAD_AVG = 0.5
}
/*
* Abstracts metrics provider that returns metrics of the system the node is running at
*/
trait MetricsProvider {
/*
* Gets metrics of the local system
*/
def getLocalMetrics: NodeMetrics
}
/*
* Loads JVM metrics through JMX monitoring beans
*/
class JMXMetricsProvider extends MetricsProvider {
import MetricsProvider._
private val memoryMXBean = ManagementFactory.getMemoryMXBean
private val osMXBean = ManagementFactory.getOperatingSystemMXBean
/*
* Validates and calculates system load average
*
* @param avg system load average obtained from a specific monitoring provider (may be incorrect)
* @return system load average, or default value(<code>0.5</code>), if passed value was out of permitted
* bounds (0.0 to 1.0)
*/
@inline
protected final def calcSystemLoadAverage(avg: Double) =
if (avg >= MIN_SYS_LOAD_AVG && avg <= MAX_SYS_LOAD_AVG) avg else DEF_SYS_LOAD_AVG
protected def systemLoadAverage = calcSystemLoadAverage(osMXBean.getSystemLoadAverage)
def getLocalMetrics =
DefaultNodeMetrics(Cluster.nodeAddress.nodeName,
memoryMXBean.getHeapMemoryUsage.getUsed,
memoryMXBean.getHeapMemoryUsage.getCommitted,
memoryMXBean.getHeapMemoryUsage.getMax,
osMXBean.getAvailableProcessors,
systemLoadAverage)
}
/*
* Loads wider range of metrics of a better quality with Hyperic Sigar (native library)
*
* @param refreshTimeout Sigar gathers metrics during this interval
*/
class SigarMetricsProvider private (private val sigarInstance: AnyRef) extends JMXMetricsProvider {
private val reportErrors = new Switch(true)
private val getCpuPercMethod = sigarInstance.getClass.getMethod("getCpuPerc")
private val sigarCpuCombinedMethod = getCpuPercMethod.getReturnType.getMethod("getCombined")
/*
* Wraps reflective calls to Hyperic Sigar
*
* @param f reflective call to Hyperic Sigar
* @param fallback function, which is invoked, if call to Sigar has been finished with exception
*/
private def callSigarMethodOrElse[T](callSigar: T, fallback: T): T =
try callSigar catch {
case thrw
reportErrors.switchOff {
EventHandler.warning(this, "Failed to get metrics from Hyperic Sigar. %s: %s"
.format(thrw.getClass.getName, thrw.getMessage))
}
fallback
}
/*
* Obtains system load average from Sigar
* If the value cannot be obtained, falls back to system load average taken from JMX
*/
override def systemLoadAverage = callSigarMethodOrElse(
calcSystemLoadAverage(sigarCpuCombinedMethod
.invoke(getCpuPercMethod.invoke(sigarInstance)).asInstanceOf[Double]),
super.systemLoadAverage)
}
object SigarMetricsProvider {
/*
* Instantiates Sigar metrics provider through reflections, in order to avoid creating dependencies to
* Hiperic Sigar library
*/
def apply(refreshTimeout: Int): Either[Throwable, MetricsProvider] = try {
for {
sigarInstance createInstance[AnyRef]("org.hyperic.sigar.Sigar", noParams, noArgs).right
sigarProxyCacheClass: Class[_] getClassFor("org.hyperic.sigar.SigarProxyCache").right
} yield new SigarMetricsProvider(sigarProxyCacheClass
.getMethod("newInstance", Array(sigarInstance.getClass, classOf[Int]): _*)
.invoke(null, sigarInstance, new java.lang.Integer(refreshTimeout)))
} catch {
case thrw Left(thrw)
}
}

View file

@ -1,366 +0,0 @@
package akka.cluster.storage
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
import akka.cluster.zookeeper.AkkaZkClient
import akka.AkkaException
import org.apache.zookeeper.{ KeeperException, CreateMode }
import org.apache.zookeeper.data.Stat
import java.util.concurrent.ConcurrentHashMap
import annotation.tailrec
import java.lang.{ RuntimeException, UnsupportedOperationException }
/**
* Simple abstraction to store an Array of bytes based on some String key.
*
* Nothing is being said about ACID, transactions etc. It depends on the implementation
* of this Storage interface of what is and isn't done on the lowest level.
*
* The amount of data that is allowed to be insert/updated is implementation specific. The InMemoryStorage
* has no limits, but the ZooKeeperStorage has a maximum size of 1 mb.
*
* TODO: Class is up for better names.
* TODO: Instead of a String as key, perhaps also a byte-array.
*/
trait Storage {
/**
* Loads the VersionedData for the given key.
*
* This call doesn't care about the actual version of the data.
*
* @param key: the key of the VersionedData to load.
* @return the VersionedData for the given entry.
* @throws MissingDataException if the entry with the given key doesn't exist.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def load(key: String): VersionedData
/**
* Loads the VersionedData for the given key and expectedVersion.
*
* This call can be used for optimistic locking since the version is included.
*
* @param key: the key of the VersionedData to load
* @param expectedVersion the version the data to load should have.
* @throws MissingDataException if the data with the given key doesn't exist.
* @throws BadVersionException if the version is not the expected version.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def load(key: String, expectedVersion: Long): VersionedData
/**
* Checks if a VersionedData with the given key exists.
*
* @param key the key to check the existence for.
* @return true if exists, false if not.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def exists(key: String): Boolean
/**
* Inserts a byte-array based on some key.
*
* @param key the key of the Data to insert.
* @param bytes the data to insert.
* @return the version of the written data (can be used for optimistic locking).
* @throws DataExistsException when VersionedData with the given Key already exists.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def insert(key: String, bytes: Array[Byte]): Long
/**
* Inserts the data if there is no data for that key, or overwrites it if it is there.
*
* This is the method you want to call if you just want to save something and don't
* care about any lost update issues.
*
* @param key the key of the data
* @param bytes the data to insert
* @return the version of the written data (can be used for optimistic locking).
* @throws StorageException if anything goes wrong while accessing the storage
*/
def insertOrOverwrite(key: String, bytes: Array[Byte]): Long
/**
* Overwrites the current data for the given key. This call doesn't care about the version of the existing data.
*
* @param key the key of the data to overwrite
* @param bytes the data to insert.
* @return the version of the written data (can be used for optimistic locking).
* @throws MissingDataException when the entry with the given key doesn't exist.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def overwrite(key: String, bytes: Array[Byte]): Long
/**
* Updates an existing value using an optimistic lock. So it expect the current data to have the expectedVersion
* and only then, it will do the update.
*
* @param key the key of the data to update
* @param bytes the content to write for the given key
* @param expectedVersion the version of the content that is expected to be there.
* @return the version of the written data (can be used for optimistic locking).
* @throws MissingDataException if no data for the given key exists
* @throws BadVersionException if the version if the found data doesn't match the expected version. So essentially
* if another update was already done.
* @throws StorageException if anything goes wrong while accessing the storage
*/
def update(key: String, bytes: Array[Byte], expectedVersion: Long): Long
}
/**
* The VersionedData is a container of data (some bytes) and a version (a Long).
*/
class VersionedData(val data: Array[Byte], val version: Long) {}
/**
* An AkkaException thrown by the Storage module.
*/
class StorageException(msg: String = null, cause: java.lang.Throwable = null) extends AkkaException(msg, cause) {
def this(msg: String) = this(msg, null);
}
/**
* *
* A StorageException thrown when an operation is done on a non existing node.
*/
class MissingDataException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause) {
def this(msg: String) = this(msg, null);
}
/**
* A StorageException thrown when an operation is done on an existing node, but no node was expected.
*/
class DataExistsException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause) {
def this(msg: String) = this(msg, null);
}
/**
* A StorageException thrown when an operation causes an optimistic locking failure.
*/
class BadVersionException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause) {
def this(msg: String) = this(msg, null);
}
/**
* A Storage implementation based on ZooKeeper.
*
* The store method is atomic:
* - so everything is written or nothing is written
* - is isolated, so threadsafe,
* but it will not participate in any transactions.
*
*/
class ZooKeeperStorage(zkClient: AkkaZkClient, root: String = "/peter/storage") extends Storage {
var path = ""
//makes sure that the complete root exists on zookeeper.
root.split("/").foreach(
item if (item.size > 0) {
path = path + "/" + item
if (!zkClient.exists(path)) {
//it could be that another thread is going to create this root node as well, so ignore it when it happens.
try {
zkClient.create(path, "".getBytes, CreateMode.PERSISTENT)
} catch {
case ignore: KeeperException.NodeExistsException
}
}
})
def toZkPath(key: String): String = {
root + "/" + key
}
def load(key: String) = try {
val stat = new Stat
val arrayOfBytes = zkClient.connection.readData(root + "/" + key, stat, false)
new VersionedData(arrayOfBytes, stat.getVersion)
} catch {
case e: KeeperException.NoNodeException throw new MissingDataException(
String.format("Failed to load key [%s]: no data was found", key), e)
case e: KeeperException throw new StorageException(
String.format("Failed to load key [%s]", key), e)
}
def load(key: String, expectedVersion: Long) = try {
val stat = new Stat
val arrayOfBytes = zkClient.connection.readData(root + "/" + key, stat, false)
if (stat.getVersion != expectedVersion) throw new BadVersionException(
"Failed to update key [" + key + "]: version mismatch, expected [" + expectedVersion + "]" +
" but found [" + stat.getVersion + "]")
new VersionedData(arrayOfBytes, stat.getVersion)
} catch {
case e: KeeperException.NoNodeException throw new MissingDataException(
String.format("Failed to load key [%s]: no data was found", key), e)
case e: KeeperException throw new StorageException(
String.format("Failed to load key [%s]", key), e)
}
def insertOrOverwrite(key: String, bytes: Array[Byte]) = {
try {
throw new UnsupportedOperationException()
} catch {
case e: KeeperException.NodeExistsException throw new DataExistsException(
String.format("Failed to insert key [%s]: an entry already exists with the same key", key), e)
case e: KeeperException throw new StorageException(
String.format("Failed to insert key [%s]", key), e)
}
}
def insert(key: String, bytes: Array[Byte]): Long = {
try {
zkClient.connection.create(root + "/" + key, bytes, CreateMode.PERSISTENT)
//todo: how to get hold of the version.
val version: Long = 0
version
} catch {
case e: KeeperException.NodeExistsException throw new DataExistsException(
String.format("Failed to insert key [%s]: an entry already exists with the same key", key), e)
case e: KeeperException throw new StorageException(
String.format("Failed to insert key [%s]", key), e)
}
}
def exists(key: String) = try {
zkClient.connection.exists(toZkPath(key), false)
} catch {
case e: KeeperException throw new StorageException(
String.format("Failed to check existance for key [%s]", key), e)
}
def update(key: String, bytes: Array[Byte], expectedVersion: Long): Long = {
try {
zkClient.connection.writeData(root + "/" + key, bytes, expectedVersion.asInstanceOf[Int])
throw new RuntimeException()
} catch {
case e: KeeperException.BadVersionException throw new BadVersionException(
String.format("Failed to update key [%s]: version mismatch", key), e)
case e: KeeperException throw new StorageException(
String.format("Failed to update key [%s]", key), e)
}
}
def overwrite(key: String, bytes: Array[Byte]): Long = {
try {
zkClient.connection.writeData(root + "/" + key, bytes)
-1L
} catch {
case e: KeeperException.NoNodeException throw new MissingDataException(
String.format("Failed to overwrite key [%s]: a previous entry already exists", key), e)
case e: KeeperException throw new StorageException(
String.format("Failed to overwrite key [%s]", key), e)
}
}
}
object InMemoryStorage {
val InitialVersion = 0;
}
/**
* An in memory {@link RawStore} implementation. Useful for testing purposes.
*/
final class InMemoryStorage extends Storage {
private val map = new ConcurrentHashMap[String, VersionedData]()
def load(key: String) = {
val result = map.get(key)
if (result == null) throw new MissingDataException(
String.format("Failed to load key [%s]: no data was found", key))
result
}
def load(key: String, expectedVersion: Long) = {
val result = load(key)
if (result.version != expectedVersion) throw new BadVersionException(
"Failed to load key [" + key + "]: version mismatch, expected [" + result.version + "] " +
"but found [" + expectedVersion + "]")
result
}
def exists(key: String) = map.containsKey(key)
def insert(key: String, bytes: Array[Byte]): Long = {
val version: Long = InMemoryStorage.InitialVersion
val result = new VersionedData(bytes, version)
val previous = map.putIfAbsent(key, result)
if (previous != null) throw new DataExistsException(
String.format("Failed to insert key [%s]: the key already has been inserted previously", key))
version
}
@tailrec
def update(key: String, bytes: Array[Byte], expectedVersion: Long): Long = {
val found = map.get(key)
if (found == null) throw new MissingDataException(
String.format("Failed to update key [%s], no previous entry exist", key))
if (expectedVersion != found.version) throw new BadVersionException(
"Failed to update key [" + key + "]: version mismatch, expected [" + expectedVersion + "]" +
" but found [" + found.version + "]")
val newVersion: Long = expectedVersion + 1
if (map.replace(key, found, new VersionedData(bytes, newVersion))) newVersion
else update(key, bytes, expectedVersion)
}
@tailrec
def overwrite(key: String, bytes: Array[Byte]): Long = {
val current = map.get(key)
if (current == null) throw new MissingDataException(
String.format("Failed to overwrite key [%s], no previous entry exist", key))
val update = new VersionedData(bytes, current.version + 1)
if (map.replace(key, current, update)) update.version
else overwrite(key, bytes)
}
def insertOrOverwrite(key: String, bytes: Array[Byte]): Long = {
val version = InMemoryStorage.InitialVersion
val result = new VersionedData(bytes, version)
val previous = map.putIfAbsent(key, result)
if (previous == null) result.version
else overwrite(key, bytes)
}
}
//TODO: To minimize the number of dependencies, should the Storage not be placed in a seperate module?
//class VoldemortRawStorage(storeClient: StoreClient) extends Storage {
//
// def load(Key: String) = {
// try {
//
// } catch {
// case
// }
// }
//
// override def insert(key: String, bytes: Array[Byte]) {
// throw new UnsupportedOperationException()
// }
//
// def update(key: String, bytes: Array[Byte]) {
// throw new UnsupportedOperationException()
// }
//}

View file

@ -1,34 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.zookeeper
import org.I0Itec.zkclient._
import org.I0Itec.zkclient.serialize._
import org.I0Itec.zkclient.exception._
/**
* ZooKeeper client. Holds the ZooKeeper connection and manages its session.
*/
class AkkaZkClient(zkServers: String,
sessionTimeout: Int,
connectionTimeout: Int,
zkSerializer: ZkSerializer = new SerializableSerializer)
extends ZkClient(zkServers, sessionTimeout, connectionTimeout, zkSerializer) {
def connection: ZkConnection = _connection.asInstanceOf[ZkConnection]
def reconnect() {
val zkLock = getEventLock
zkLock.lock()
try {
_connection.close()
_connection.connect(this)
} catch {
case e: InterruptedException throw new ZkInterruptedException(e)
} finally {
zkLock.unlock()
}
}
}

View file

@ -1,32 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.zookeeper
import org.I0Itec.zkclient._
import org.apache.commons.io.FileUtils
import java.io.File
object AkkaZooKeeper {
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalServer(dataPath: String, logPath: String): ZkServer =
startLocalServer(dataPath, logPath, 2181, 500)
/**
* Starts up a local ZooKeeper server. Should only be used for testing purposes.
*/
def startLocalServer(dataPath: String, logPath: String, port: Int, tickTime: Int): ZkServer = {
FileUtils.deleteDirectory(new File(dataPath))
FileUtils.deleteDirectory(new File(logPath))
val zkServer = new ZkServer(
dataPath, logPath,
new IDefaultNameSpace() {
def createDefaultNameSpace(zkClient: ZkClient) {}
},
port, tickTime)
zkServer.start()
zkServer
}
}

View file

@ -1,104 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.zookeeper
import akka.util.Duration
import akka.util.duration._
import org.I0Itec.zkclient._
import org.I0Itec.zkclient.exception._
import java.util.{ List JList }
import java.util.concurrent.CountDownLatch
class BarrierTimeoutException(message: String) extends RuntimeException(message)
/**
* Barrier based on Zookeeper barrier tutorial.
*/
object ZooKeeperBarrier {
val BarriersNode = "/barriers"
val DefaultTimeout = 60 seconds
def apply(zkClient: ZkClient, name: String, node: String, count: Int) =
new ZooKeeperBarrier(zkClient, name, node, count, DefaultTimeout)
def apply(zkClient: ZkClient, name: String, node: String, count: Int, timeout: Duration) =
new ZooKeeperBarrier(zkClient, name, node, count, timeout)
def apply(zkClient: ZkClient, cluster: String, name: String, node: String, count: Int) =
new ZooKeeperBarrier(zkClient, cluster + "-" + name, node, count, DefaultTimeout)
def apply(zkClient: ZkClient, cluster: String, name: String, node: String, count: Int, timeout: Duration) =
new ZooKeeperBarrier(zkClient, cluster + "-" + name, node, count, timeout)
def ignore[E: Manifest](body: Unit) {
try {
body
} catch {
case e if manifest[E].erasure.isAssignableFrom(e.getClass) ()
}
}
}
/**
* Barrier based on Zookeeper barrier tutorial.
*/
class ZooKeeperBarrier(zkClient: ZkClient, name: String, node: String, count: Int, timeout: Duration)
extends IZkChildListener {
import ZooKeeperBarrier.{ BarriersNode, ignore }
val barrier = BarriersNode + "/" + name
val entry = barrier + "/" + node
val ready = barrier + "/ready"
val exitBarrier = new CountDownLatch(1)
ignore[ZkNodeExistsException](zkClient.createPersistent(BarriersNode))
ignore[ZkNodeExistsException](zkClient.createPersistent(barrier))
def apply(body: Unit) {
enter()
body
leave()
}
/**
* An await does a enter/leave making this barrier a 'single' barrier instead of a double barrier.
*/
def await() {
enter()
leave()
}
def enter() = {
zkClient.createEphemeral(entry)
if (zkClient.countChildren(barrier) >= count)
ignore[ZkNodeExistsException](zkClient.createPersistent(ready))
else
zkClient.waitUntilExists(ready, timeout.unit, timeout.length)
if (!zkClient.exists(ready)) {
throw new BarrierTimeoutException("Timeout (%s) while waiting for entry barrier" format timeout)
}
zkClient.subscribeChildChanges(barrier, this)
}
def leave() {
zkClient.delete(entry)
exitBarrier.await(timeout.length, timeout.unit)
if (zkClient.countChildren(barrier) > 0) {
zkClient.unsubscribeChildChanges(barrier, this)
throw new BarrierTimeoutException("Timeout (%s) while waiting for exit barrier" format timeout)
}
zkClient.unsubscribeChildChanges(barrier, this)
}
def handleChildChange(path: String, children: JList[String]) {
if (children.size <= 1) {
ignore[ZkNoNodeException](zkClient.delete(ready))
exitBarrier.countDown()
}
}
}

View file

@ -1,4 +1,4 @@
// package akka.remote
// package akka.cluster
// import akka.actor.Actor
// import akka.remote._

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1 +0,0 @@
-Dakka.cluster.nodename=node1 -Dakka.remote.port=9991

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1 +0,0 @@
-Dakka.cluster.nodename=node2 -Dakka.remote.port=9992

View file

@ -1,63 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.api.changelisteners.newleader
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll
import akka.cluster._
import ChangeListener._
import Cluster._
import akka.cluster.LocalCluster._
import java.util.concurrent._
object NewLeaderChangeListenerMultiJvmSpec {
var NrOfNodes = 2
}
class NewLeaderChangeListenerMultiJvmNode1 extends MasterClusterTestNode {
import NewLeaderChangeListenerMultiJvmSpec._
val testNodes = NrOfNodes
"A NewLeader change listener" must {
"be invoked after leader election is completed" ignore {
barrier("start-node1", NrOfNodes) {
Cluster.node.start()
}
barrier("start-node2", NrOfNodes).await()
System.exit(0)
}
}
}
class NewLeaderChangeListenerMultiJvmNode2 extends ClusterTestNode {
import NewLeaderChangeListenerMultiJvmSpec._
"A NewLeader change listener" must {
"be invoked after leader election is completed" ignore {
val latch = new CountDownLatch(1)
barrier("start-node1", NrOfNodes).await()
barrier("start-node2", NrOfNodes) {
node.register(new ChangeListener {
override def newLeader(node: String, client: ClusterNode) {
latch.countDown
}
})
}
latch.await(10, TimeUnit.SECONDS) must be === true
node.shutdown()
}
}
}

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1,65 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.api.changelisteners.nodeconnected
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll
import akka.cluster._
import ChangeListener._
import Cluster._
import akka.cluster.LocalCluster._
import java.util.concurrent._
object NodeConnectedChangeListenerMultiJvmSpec {
var NrOfNodes = 2
}
class NodeConnectedChangeListenerMultiJvmNode1 extends MasterClusterTestNode {
import NodeConnectedChangeListenerMultiJvmSpec._
val testNodes = NrOfNodes
"A NodeConnected change listener" must {
"be invoked when a new node joins the cluster" in {
val latch = new CountDownLatch(1)
node.register(new ChangeListener {
override def nodeConnected(node: String, client: ClusterNode) {
latch.countDown
}
})
barrier("start-node1", NrOfNodes) {
Cluster.node.start()
}
barrier("start-node2", NrOfNodes) {
latch.await(5, TimeUnit.SECONDS) must be === true
}
node.shutdown()
}
}
}
class NodeConnectedChangeListenerMultiJvmNode2 extends ClusterTestNode {
import NodeConnectedChangeListenerMultiJvmSpec._
"A NodeConnected change listener" must {
"be invoked when a new node joins the cluster" in {
barrier("start-node1", NrOfNodes).await()
barrier("start-node2", NrOfNodes) {
Cluster.node.start()
}
node.shutdown()
}
}
}

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1,65 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.api.changelisteners.nodedisconnected
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterAll
import akka.cluster._
import ChangeListener._
import Cluster._
import akka.cluster.LocalCluster._
import java.util.concurrent._
object NodeDisconnectedChangeListenerMultiJvmSpec {
var NrOfNodes = 2
}
class NodeDisconnectedChangeListenerMultiJvmNode1 extends MasterClusterTestNode {
import NodeDisconnectedChangeListenerMultiJvmSpec._
val testNodes = NrOfNodes
"A NodeDisconnected change listener" must {
"be invoked when a new node leaves the cluster" in {
val latch = new CountDownLatch(1)
node.register(new ChangeListener {
override def nodeDisconnected(node: String, client: ClusterNode) {
latch.countDown
}
})
barrier("start-node1", NrOfNodes) {
Cluster.node.start()
}
barrier("start-node2", NrOfNodes).await()
latch.await(10, TimeUnit.SECONDS) must be === true
node.shutdown()
}
}
}
class NodeDisconnectedChangeListenerMultiJvmNode2 extends ClusterTestNode {
import NodeDisconnectedChangeListenerMultiJvmSpec._
"A NodeDisconnected change listener" must {
"be invoked when a new node leaves the cluster" in {
barrier("start-node1", NrOfNodes).await()
barrier("start-node2", NrOfNodes) {
Cluster.node.start()
}
node.shutdown()
}
}
}

View file

@ -1,2 +0,0 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"

View file

@ -1 +0,0 @@
-Dakka.cluster.nodename=node1 -Dakka.remote.port=9991

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